diff --git a/.github/workflows/bug-closed.yml b/.github/workflows/bug-closed.yml deleted file mode 100644 index 36e8574f00e19..0000000000000 --- a/.github/workflows/bug-closed.yml +++ /dev/null @@ -1,28 +0,0 @@ -name: Bug Closed - -on: - issues: - types: - - closed - -jobs: - label_issues: - if: | - contains(github.event.issue.labels.*.name, 'type/bug') && - !(contains(join(github.event.issue.labels.*.name, ', '), 'affects-') && - contains(join(github.event.issue.labels.*.name, ', '), 'fixes-')) - runs-on: ubuntu-latest - permissions: - issues: write - steps: - - name: Label issues - uses: andymckay/labeler@1.0.3 - with: - add-labels: "needs-more-info" - repo-token: ${{ secrets.GITHUB_TOKEN }} - - name: Add comment - uses: peter-evans/create-or-update-comment@v1.4.5 - with: - issue-number: ${{ github.event.issue.number }} - body: | - Please check whether the issue should be labeled with 'affects-x.y' or 'fixes-x.y.z', and then remove 'needs-more-info' label. diff --git a/.golangci.yml b/.golangci.yml index 9f545dc9f2c23..ce37ff0de15b3 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -24,6 +24,7 @@ linters: - makezero - durationcheck - prealloc + - predeclared linters-settings: staticcheck: diff --git a/Makefile b/Makefile index 58af99d25f450..4e0c2f578535e 100644 --- a/Makefile +++ b/Makefile @@ -14,7 +14,7 @@ include Makefile.common -.PHONY: all clean test gotest server dev benchkv benchraw check checklist parser tidy ddltest build_br build_lightning build_lightning-ctl build_dumpling +.PHONY: all clean test gotest server dev benchkv benchraw check checklist parser tidy ddltest build_br build_lightning build_lightning-ctl build_dumpling ut default: server buildsucc @@ -28,7 +28,7 @@ all: dev server benchkv parser: @echo "remove this command later, when our CI script doesn't call it" -dev: checklist check explaintest devgotest gogenerate br_unit_test test_part_parser_dev +dev: checklist check explaintest gogenerate br_unit_test test_part_parser_dev @>&2 echo "Great, all tests passed." # Install the check tools. @@ -115,19 +115,12 @@ explaintest: server_check ddltest: @cd cmd/ddltest && $(GO) test -o ../../bin/ddltest -c -devgotest: failpoint-enable -# grep regex: Filter out all tidb logs starting with: -# - '[20' (like [2021/09/15 ...] [INFO]..) -# - 'PASS:' to ignore passed tests -# - 'ok ' to ignore passed directories - @echo "Running in native mode." - @export log_level=info; export TZ='Asia/Shanghai'; \ - $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -cover $(PACKAGES_TIDB_TESTS) -check.p true > gotest.log || { $(FAILPOINT_DISABLE); grep -v '^\([[]20\|PASS:\|ok \)' 'gotest.log'; exit 1; } - @$(FAILPOINT_DISABLE) +CLEAN_UT_BINARY := find . -name '*.test.bin'| xargs rm -ut: failpoint-enable tools/bin/ut - tools/bin/ut $(X); +ut: tools/bin/ut tools/bin/xprog failpoint-enable + tools/bin/ut $(X) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) + @$(CLEAN_UT_BINARY) gotest: failpoint-enable @echo "Running in native mode." @@ -135,14 +128,21 @@ gotest: failpoint-enable $(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -timeout 20m -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; } @$(FAILPOINT_DISABLE) -gotest_in_verify_ci: failpoint-enable tools/bin/gotestsum +gotest_in_verify_ci: tools/bin/xprog tools/bin/ut failpoint-enable @echo "Running gotest_in_verify_ci" @mkdir -p $(TEST_COVERAGE_DIR) @export TZ='Asia/Shanghai'; \ - CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) \ - -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \ - $(PACKAGES_TIDB_TESTS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; } + tools/bin/ut --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" --coverprofile "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" --except unstable.txt || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) + @$(CLEAN_UT_BINARY) + +gotest_unstable_in_verify_ci: tools/bin/xprog tools/bin/ut failpoint-enable + @echo "Running gotest_in_verify_ci" + @mkdir -p $(TEST_COVERAGE_DIR) + @export TZ='Asia/Shanghai'; \ + tools/bin/ut --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" --coverprofile "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" --only unstable.txt || { $(FAILPOINT_DISABLE); exit 1; } + @$(FAILPOINT_DISABLE) + @$(CLEAN_UT_BINARY) race: failpoint-enable @export log_level=debug; \ @@ -216,6 +216,10 @@ tools/bin/ut: tools/check/ut.go cd tools/check; \ $(GO) build -o ../bin/ut ut.go +tools/bin/xprog: tools/check/xprog.go + cd tools/check; \ + $(GO) build -o ../bin/xprog xprog.go + tools/bin/megacheck: tools/check/go.mod cd tools/check; \ $(GO) build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck @@ -421,3 +425,6 @@ dumpling_bins: tools/bin/gotestsum: tools/check/go.mod cd tools/check && $(GO) build -o ../bin/gotestsum gotest.tools/gotestsum + +generate_grafana_scripts: + @cd metrics/grafana && mv tidb_summary.json tidb_summary.json.committed && ./generate_json.sh && diff -u tidb_summary.json.committed tidb_summary.json && rm tidb_summary.json.committed diff --git a/bindinfo/capture_test.go b/bindinfo/capture_test.go index 1509213d479a9..e86157a9191d2 100644 --- a/bindinfo/capture_test.go +++ b/bindinfo/capture_test.go @@ -555,6 +555,62 @@ func TestIssue25505(t *testing.T) { } } +func TestCaptureUserFilter(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + stmtsummary.StmtSummaryByDigestMap.Clear() + tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = on") + defer func() { + tk.MustExec("SET GLOBAL tidb_capture_plan_baselines = off") + }() + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk.MustExec("select * from t where a > 10") + tk.MustExec("select * from t where a > 10") + tk.MustExec("admin capture bindings") + rows := tk.MustQuery("show global bindings").Rows() + require.Len(t, rows, 1) + require.Equal(t, "select * from `test` . `t` where `a` > ?", rows[0][0]) + + // test user filter + utilCleanBindingEnv(tk, dom) + stmtsummary.StmtSummaryByDigestMap.Clear() + tk.MustExec("insert into mysql.capture_plan_baselines_blacklist(filter_type, filter_value) values('user', 'root')") + tk.MustExec("select * from t where a > 10") + tk.MustExec("select * from t where a > 10") + tk.MustExec("admin capture bindings") + rows = tk.MustQuery("show global bindings").Rows() + require.Len(t, rows, 0) // cannot capture the stmt + + // change another user + tk.MustExec(`create user usr1`) + tk.MustExec(`grant all on *.* to usr1 with grant option`) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + require.True(t, tk2.Session().Auth(&auth.UserIdentity{Username: "usr1", Hostname: "%"}, nil, nil)) + tk2.MustExec("select * from t where a > 10") + tk2.MustExec("select * from t where a > 10") + tk2.MustExec("admin capture bindings") + rows = tk2.MustQuery("show global bindings").Rows() + require.Len(t, rows, 1) // can capture the stmt + + // use user-filter with other types of filter together + utilCleanBindingEnv(tk, dom) + stmtsummary.StmtSummaryByDigestMap.Clear() + tk.MustExec("insert into mysql.capture_plan_baselines_blacklist(filter_type, filter_value) values('user', 'root')") + tk.MustExec("insert into mysql.capture_plan_baselines_blacklist(filter_type, filter_value) values('table', 'test.t')") + tk2.MustExec("select * from t where a > 10") + tk2.MustExec("select * from t where a > 10") + tk2.MustExec("admin capture bindings") + rows = tk2.MustQuery("show global bindings").Rows() + require.Len(t, rows, 0) // filtered by the table filter +} + func TestCaptureFilter(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() diff --git a/bindinfo/handle.go b/bindinfo/handle.go index d57d2edb1da0f..5653d82ef2391 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -131,15 +131,11 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { } exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source - FROM mysql.bind_info WHERE update_time > %? ORDER BY update_time, create_time`, updateTime) - if err != nil { - return err - } - // No need to acquire the session context lock for ExecRestrictedStmt, it + // No need to acquire the session context lock for ExecRestrictedSQL, it // uses another background session. - rows, _, err := exec.ExecRestrictedStmt(context.Background(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, `SELECT original_sql, bind_sql, default_db, status, create_time, update_time, charset, collation, source + FROM mysql.bind_info WHERE update_time > %? ORDER BY update_time, create_time`, updateTime) if err != nil { h.bindInfo.Unlock() @@ -661,6 +657,7 @@ type captureFilter struct { dbs map[string]struct{} frequency int64 tables map[stmtctx.TableEntry]struct{} + users map[string]struct{} fail bool currentDB string @@ -690,7 +687,7 @@ func (cf *captureFilter) Leave(in ast.Node) (out ast.Node, ok bool) { } func (cf *captureFilter) isEmpty() bool { - return len(cf.dbs) == 0 && len(cf.tables) == 0 + return len(cf.dbs) == 0 && len(cf.tables) == 0 && len(cf.users) == 0 } func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) { @@ -698,16 +695,12 @@ func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) { dbs: make(map[string]struct{}), frequency: 1, tables: make(map[stmtctx.TableEntry]struct{}), + users: make(map[string]struct{}), } exec := h.sctx.Context.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`) - if err != nil { - logutil.BgLogger().Warn("[sql-bind] failed to parse query for mysql.capture_plan_baselines_blacklist load", zap.Error(err)) - return - } - // No need to acquire the session context lock for ExecRestrictedStmt, it + // No need to acquire the session context lock for ExecRestrictedSQL, it // uses another background session. - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, `SELECT filter_type, filter_value FROM mysql.capture_plan_baselines_blacklist order by filter_type`) if err != nil { logutil.BgLogger().Warn("[sql-bind] failed to load mysql.capture_plan_baselines_blacklist", zap.Error(err)) return @@ -729,6 +722,8 @@ func (h *BindHandle) extractCaptureFilterFromStorage() (filter *captureFilter) { Table: strs[1], } filter.tables[tblEntry] = struct{}{} + case "user": + filter.users[valStr] = struct{}{} case "frequency": f, err := strconv.ParseInt(valStr, 10, 64) if err != nil { @@ -771,6 +766,19 @@ func (h *BindHandle) CaptureBaselines() { if captureFilter.fail { continue } + + if len(captureFilter.users) > 0 { + filteredByUser := true + for user := range bindableStmt.Users { + if _, ok := captureFilter.users[user]; !ok { + filteredByUser = false // some user not in the black-list has processed this stmt + break + } + } + if filteredByUser { + continue + } + } } dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema) normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query)) @@ -926,9 +934,9 @@ func (h *BindHandle) SaveEvolveTasksToStore() { } func getEvolveParameters(ctx sessionctx.Context) (time.Duration, time.Time, time.Time, error) { - stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams( + rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL( context.TODO(), - true, + nil, "SELECT variable_name, variable_value FROM mysql.global_variables WHERE variable_name IN (%?, %?, %?)", variable.TiDBEvolvePlanTaskMaxTime, variable.TiDBEvolvePlanTaskStartTime, @@ -937,10 +945,6 @@ func getEvolveParameters(ctx sessionctx.Context) (time.Duration, time.Time, time if err != nil { return 0, time.Time{}, time.Time{}, err } - rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(context.TODO(), stmt) - if err != nil { - return 0, time.Time{}, time.Time{}, err - } maxTime, startTimeStr, endTimeStr := int64(variable.DefTiDBEvolvePlanTaskMaxTime), variable.DefTiDBEvolvePlanTaskStartTime, variable.DefAutoAnalyzeEndTime for _, row := range rows { switch row.GetString(0) { diff --git a/bindinfo/handle_test.go b/bindinfo/handle_test.go index 8522d95902c1c..7a5476d5d2947 100644 --- a/bindinfo/handle_test.go +++ b/bindinfo/handle_test.go @@ -38,7 +38,7 @@ func utilCleanBindingEnv(tk *testkit.TestKit, dom *domain.Domain) { func utilNormalizeWithDefaultDB(t *testing.T, sql, db string) (string, string) { testParser := parser.New() stmt, err := testParser.ParseOneStmt(sql, "", "") - require.Nil(t, err) + require.NoError(t, err) normalized, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, "test", "")) return normalized, digest.String() } @@ -82,7 +82,7 @@ func TestBindingLastUpdateTime(t *testing.T) { bindHandle := bindinfo.NewBindHandle(tk.Session()) err := bindHandle.Update(true) - require.Nil(t, err) + require.NoError(t, err) sql, hash := parser.NormalizeDigest("select * from test . t0") bindData := bindHandle.GetBindRecord(hash.String(), sql, "test") require.Equal(t, 1, len(bindData.Bindings)) @@ -121,7 +121,7 @@ func TestBindingLastUpdateTimeWithInvalidBind(t *testing.T) { require.Equal(t, updateTime1, "2000-01-01 09:00:00.000") rows2 := tk.MustQuery("show global bindings").Rows() - require.Equal(t, len(rows2), 0) + require.Len(t, rows2, 0) } func TestBindParse(t *testing.T) { @@ -146,7 +146,7 @@ func TestBindParse(t *testing.T) { tk.MustExec(sql) bindHandle := bindinfo.NewBindHandle(tk.Session()) err := bindHandle.Update(true) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, bindHandle.Size()) sql, hash := parser.NormalizeDigest("select * from test . t") @@ -162,7 +162,7 @@ func TestBindParse(t *testing.T) { require.NotNil(t, bind.CreateTime) require.NotNil(t, bind.UpdateTime) dur, err := bind.SinceUpdateTime() - require.Nil(t, err) + require.NoError(t, err) require.GreaterOrEqual(t, int64(dur), int64(0)) // Test fields with quotes or slashes. @@ -387,19 +387,19 @@ func TestGlobalBinding(t *testing.T) { metrics.BindMemoryUsage.Reset() _, err := tk.Exec("create global " + testSQL.createSQL) - require.Nil(t, err, "err %v", err) + require.NoError(t, err, "err %v", err) if testSQL.overlaySQL != "" { _, err = tk.Exec("create global " + testSQL.overlaySQL) - require.Nil(t, err) + require.NoError(t, err) } pb := &dto.Metric{} err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, float64(1), pb.GetGauge().GetValue()) err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, testSQL.memoryUsage, pb.GetGauge().GetValue()) sql, hash := utilNormalizeWithDefaultDB(t, testSQL.querySQL, "test") @@ -417,10 +417,10 @@ func TestGlobalBinding(t *testing.T) { require.NotNil(t, bind.UpdateTime) rs, err := tk.Exec("show global bindings") - require.Nil(t, err) + require.NoError(t, err) chk := rs.NewChunk(nil) err = rs.Next(context.TODO(), chk) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, chk.NumRows()) row := chk.GetRow(0) require.Equal(t, testSQL.originSQL, row.GetString(0)) @@ -434,7 +434,7 @@ func TestGlobalBinding(t *testing.T) { bindHandle := bindinfo.NewBindHandle(tk.Session()) err = bindHandle.Update(true) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, bindHandle.Size()) bindData = bindHandle.GetBindRecord(hash, sql, "test") @@ -450,35 +450,35 @@ func TestGlobalBinding(t *testing.T) { require.NotNil(t, bind.UpdateTime) _, err = tk.Exec("drop global " + testSQL.dropSQL) - require.Nil(t, err) + require.NoError(t, err) bindData = dom.BindHandle().GetBindRecord(hash, sql, "test") require.Nil(t, bindData) err = metrics.BindTotalGauge.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, float64(0), pb.GetGauge().GetValue()) err = metrics.BindMemoryUsage.WithLabelValues(metrics.ScopeGlobal, bindinfo.Using).Write(pb) - require.Nil(t, err) + require.NoError(t, err) // From newly created global bind handle. require.Equal(t, testSQL.memoryUsage, pb.GetGauge().GetValue()) bindHandle = bindinfo.NewBindHandle(tk.Session()) err = bindHandle.Update(true) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 0, bindHandle.Size()) bindData = bindHandle.GetBindRecord(hash, sql, "test") require.Nil(t, bindData) rs, err = tk.Exec("show global bindings") - require.Nil(t, err) + require.NoError(t, err) chk = rs.NewChunk(nil) err = rs.Next(context.TODO(), chk) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 0, chk.NumRows()) _, err = tk.Exec("delete from mysql.bind_info where source != 'builtin'") - require.Nil(t, err) + require.NoError(t, err) } } diff --git a/bindinfo/main_test.go b/bindinfo/main_test.go index 6104329761360..a2e209ec2dd0b 100644 --- a/bindinfo/main_test.go +++ b/bindinfo/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/bindinfo/session_handle_test.go b/bindinfo/session_handle_test.go index 32219063ab43a..cb908a909b5e7 100644 --- a/bindinfo/session_handle_test.go +++ b/bindinfo/session_handle_test.go @@ -106,7 +106,7 @@ func TestSessionBinding(t *testing.T) { metrics.BindMemoryUsage.Reset() _, err := tk.Exec("create session " + testSQL.createSQL) - require.Nil(t, err, "err %v", err) + require.NoError(t, err, "err %v", err) if testSQL.overlaySQL != "" { _, err = tk.Exec("create session " + testSQL.overlaySQL) diff --git a/br/cmd/tidb-lightning/main.go b/br/cmd/tidb-lightning/main.go index 9bb00ff38cc6d..d70973747eac4 100644 --- a/br/cmd/tidb-lightning/main.go +++ b/br/cmd/tidb-lightning/main.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/pingcap/tidb/br/pkg/lightning/web" "go.uber.org/zap" ) @@ -78,6 +79,9 @@ func main() { fmt.Fprintln(os.Stderr, "failed to start HTTP server:", err) return } + if len(globalCfg.App.StatusAddr) > 0 { + web.EnableCurrentProgress() + } err = func() error { if globalCfg.App.ServerMode { diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index c8cf668c04010..7d1d9e1390595 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" filter "github.com/pingcap/tidb-tools/pkg/table-filter" @@ -82,8 +83,9 @@ type Client struct { mgr ClientMgr clusterID uint64 - storage storage.ExternalStorage - backend *backuppb.StorageBackend + storage storage.ExternalStorage + backend *backuppb.StorageBackend + apiVersion kvrpcpb.APIVersion gcTTL int64 } @@ -193,6 +195,16 @@ func (bc *Client) GetClusterID() uint64 { return bc.clusterID } +// GetApiVersion sets api version of the TiKV storage +func (bc *Client) GetApiVersion() kvrpcpb.APIVersion { + return bc.apiVersion +} + +// SetApiVersion sets api version of the TiKV storage +func (bc *Client) SetApiVersion(v kvrpcpb.APIVersion) { + bc.apiVersion = v +} + // CheckBackupStorageIsLocked checks whether backups is locked. // which means we found other backup progress already write // some data files into the same backup directory or cloud prefix. @@ -707,6 +719,8 @@ func (bc *Client) fineGrainedBackup( logutil.Key("fine-grained-range-end", resp.EndKey), ) rangeTree.Put(resp.StartKey, resp.EndKey, resp.Files) + apiVersion := resp.ApiVersion + bc.SetApiVersion(apiVersion) // Update progress progressCallBack(RegionUnit) @@ -887,6 +901,69 @@ func (bc *Client) handleFineGrained( return backoffMill, nil } +func doSendBackup( + ctx context.Context, + client backuppb.BackupClient, + req backuppb.BackupRequest, + respFn func(*backuppb.BackupResponse) error, +) error { + failpoint.Inject("hint-backup-start", func(v failpoint.Value) { + logutil.CL(ctx).Info("failpoint hint-backup-start injected, " + + "process will notify the shell.") + if sigFile, ok := v.(string); ok { + file, err := os.Create(sigFile) + if err != nil { + log.Warn("failed to create file for notifying, skipping notify", zap.Error(err)) + } + if file != nil { + file.Close() + } + } + time.Sleep(3 * time.Second) + }) + bCli, err := client.Backup(ctx, &req) + failpoint.Inject("reset-retryable-error", func(val failpoint.Value) { + if val.(bool) { + logutil.CL(ctx).Debug("failpoint reset-retryable-error injected.") + err = status.Error(codes.Unavailable, "Unavailable error") + } + }) + failpoint.Inject("reset-not-retryable-error", func(val failpoint.Value) { + if val.(bool) { + logutil.CL(ctx).Debug("failpoint reset-not-retryable-error injected.") + err = status.Error(codes.Unknown, "Your server was haunted hence doesn't work, meow :3") + } + }) + if err != nil { + return err + } + defer func() { + _ = bCli.CloseSend() + }() + + for { + resp, err := bCli.Recv() + if err != nil { + if errors.Cause(err) == io.EOF { // nolint:errorlint + logutil.CL(ctx).Debug("backup streaming finish", + logutil.Key("backup-start-key", req.GetStartKey()), + logutil.Key("backup-end-key", req.GetEndKey())) + return nil + } + return err + } + // TODO: handle errors in the resp. + logutil.CL(ctx).Debug("range backed up", + logutil.Key("small-range-start-key", resp.GetStartKey()), + logutil.Key("small-range-end-key", resp.GetEndKey()), + zap.Int("api-version", int(resp.ApiVersion))) + err = respFn(resp) + if err != nil { + return errors.Trace(err) + } + } +} + // SendBackup send backup request to the given store. // Stop receiving response if respFn returns error. func SendBackup( @@ -908,40 +985,15 @@ func SendBackup( } var errReset error -backupLoop: + var errBackup error + for retry := 0; retry < backupRetryTimes; retry++ { logutil.CL(ctx).Info("try backup", zap.Int("retry time", retry), ) - failpoint.Inject("hint-backup-start", func(v failpoint.Value) { - logutil.CL(ctx).Info("failpoint hint-backup-start injected, " + - "process will notify the shell.") - if sigFile, ok := v.(string); ok { - file, err := os.Create(sigFile) - if err != nil { - log.Warn("failed to create file for notifying, skipping notify", zap.Error(err)) - } - if file != nil { - file.Close() - } - } - time.Sleep(3 * time.Second) - }) - bcli, err := client.Backup(ctx, &req) - failpoint.Inject("reset-retryable-error", func(val failpoint.Value) { - if val.(bool) { - logutil.CL(ctx).Debug("failpoint reset-retryable-error injected.") - err = status.Error(codes.Unavailable, "Unavailable error") - } - }) - failpoint.Inject("reset-not-retryable-error", func(val failpoint.Value) { - if val.(bool) { - logutil.CL(ctx).Debug("failpoint reset-not-retryable-error injected.") - err = status.Error(codes.Unknown, "Your server was haunted hence doesn't work, meow :3") - } - }) - if err != nil { - if isRetryableError(err) { + errBackup = doSendBackup(ctx, client, req, respFn) + if errBackup != nil { + if isRetryableError(errBackup) { time.Sleep(3 * time.Second) client, errReset = resetFn() if errReset != nil { @@ -950,45 +1002,11 @@ backupLoop: } continue } - logutil.CL(ctx).Error("fail to backup", zap.Uint64("StoreID", storeID), - zap.Int("retry time", retry)) - return berrors.ErrFailedToConnect.Wrap(err).GenWithStack("failed to create backup stream to store %d", storeID) - } - - for { - resp, err := bcli.Recv() - if err != nil { - if errors.Cause(err) == io.EOF { // nolint:errorlint - logutil.CL(ctx).Info("backup streaming finish", - zap.Int("retry-time", retry)) - _ = bcli.CloseSend() - break backupLoop - } - if isRetryableError(err) { - time.Sleep(3 * time.Second) - // current tikv is unavailable - client, errReset = resetFn() - if errReset != nil { - _ = bcli.CloseSend() - return errors.Annotatef(errReset, "failed to reset recv connection on store:%d "+ - "please check the tikv status", storeID) - } - _ = bcli.CloseSend() - break - } - _ = bcli.CloseSend() - return berrors.ErrFailedToConnect.Wrap(err).GenWithStack("failed to connect to store: %d with retry times:%d", storeID, retry) - } - - // TODO: handle errors in the resp. - logutil.CL(ctx).Info("range backed up", - logutil.Key("small-range-start-key", resp.GetStartKey()), - logutil.Key("small-range-end-key", resp.GetEndKey())) - err = respFn(resp) - if err != nil { - _ = bcli.CloseSend() - return errors.Trace(err) - } + logutil.CL(ctx).Error("fail to backup", zap.Uint64("StoreID", storeID), zap.Int("retry", retry)) + return berrors.ErrFailedToConnect.Wrap(errBackup).GenWithStack("failed to create backup stream to store %d", storeID) + } else { + // finish backup + break } } return nil diff --git a/br/pkg/backup/client_test.go b/br/pkg/backup/client_test.go index e4799b3192c33..19a0d0b78f2ae 100644 --- a/br/pkg/backup/client_test.go +++ b/br/pkg/backup/client_test.go @@ -45,7 +45,7 @@ type testBackup struct { } func createBackupSuite(t *testing.T) (s *testBackup, clean func()) { - _, _, pdClient, err := testutils.NewMockTiKV("", nil) + tikvClient, _, pdClient, err := testutils.NewMockTiKV("", nil) require.NoError(t, err) s = new(testBackup) s.mockPDClient = pdClient @@ -66,6 +66,8 @@ func createBackupSuite(t *testing.T) (s *testBackup, clean func()) { clean = func() { mockMgr.Close() s.cluster.Stop() + tikvClient.Close() + pdClient.Close() } return } diff --git a/br/pkg/backup/main_test.go b/br/pkg/backup/main_test.go index ee9d4be00aca6..29a95f2821f8a 100644 --- a/br/pkg/backup/main_test.go +++ b/br/pkg/backup/main_test.go @@ -25,12 +25,8 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), - goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).compactionError"), - goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mCompaction"), goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain"), - goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).tCompaction"), - goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb/util.(*BufferPool).drain"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/br/pkg/backup/push.go b/br/pkg/backup/push.go index c365eaaa96921..5a3a5aa6e426e 100644 --- a/br/pkg/backup/push.go +++ b/br/pkg/backup/push.go @@ -41,11 +41,11 @@ func (r responseAndStore) GetStore() *metapb.Store { } // newPushDown creates a push down backup. -func newPushDown(mgr ClientMgr, cap int) *pushDown { +func newPushDown(mgr ClientMgr, capacity int) *pushDown { return &pushDown{ mgr: mgr, - respCh: make(chan responseAndStore, cap), - errCh: make(chan error, cap), + respCh: make(chan responseAndStore, capacity), + errCh: make(chan error, capacity), } } diff --git a/br/pkg/checksum/main_test.go b/br/pkg/checksum/main_test.go index dc5f01b826998..04675bb970144 100644 --- a/br/pkg/checksum/main_test.go +++ b/br/pkg/checksum/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/br/pkg/conn/conn.go b/br/pkg/conn/conn.go index 67db05219e510..9ddba2a7ed3cc 100755 --- a/br/pkg/conn/conn.go +++ b/br/pkg/conn/conn.go @@ -90,10 +90,10 @@ func (p *Pool) Get(ctx context.Context) (*grpc.ClientConn, error) { } // NewConnPool creates a new Pool by the specified conn factory function and capacity. -func NewConnPool(cap int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *Pool { +func NewConnPool(capacity int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *Pool { return &Pool{ - cap: cap, - conns: make([]*grpc.ClientConn, 0, cap), + cap: capacity, + conns: make([]*grpc.ClientConn, 0, capacity), newConn: newConn, mu: sync.Mutex{}, diff --git a/br/pkg/conn/main_test.go b/br/pkg/conn/main_test.go index 7b46a892be79d..654909595fc8a 100644 --- a/br/pkg/conn/main_test.go +++ b/br/pkg/conn/main_test.go @@ -23,7 +23,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/br/pkg/kv/checksum.go b/br/pkg/kv/checksum.go deleted file mode 100644 index dbfed41aa0edc..0000000000000 --- a/br/pkg/kv/checksum.go +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "fmt" - "hash/crc64" - - "go.uber.org/zap/zapcore" -) - -var ecmaTable = crc64.MakeTable(crc64.ECMA) - -// Checksum represents the field needs checksum. -type Checksum struct { - bytes uint64 - kvs uint64 - checksum uint64 -} - -// NewKVChecksum creates Checksum. -func NewKVChecksum(checksum uint64) *Checksum { - return &Checksum{ - checksum: checksum, - } -} - -// MakeKVChecksum creates Checksum. -func MakeKVChecksum(bytes uint64, kvs uint64, checksum uint64) Checksum { - return Checksum{ - bytes: bytes, - kvs: kvs, - checksum: checksum, - } -} - -// UpdateOne add kv with its values. -func (c *Checksum) UpdateOne(kv Pair) { - sum := crc64.Update(0, ecmaTable, kv.Key) - sum = crc64.Update(sum, ecmaTable, kv.Val) - - c.bytes += uint64(len(kv.Key) + len(kv.Val)) - c.kvs++ - c.checksum ^= sum -} - -// Update add batch of kvs with their values. -func (c *Checksum) Update(kvs []Pair) { - var ( - checksum uint64 - sum uint64 - kvNum int - bytes int - ) - - for _, pair := range kvs { - sum = crc64.Update(0, ecmaTable, pair.Key) - sum = crc64.Update(sum, ecmaTable, pair.Val) - checksum ^= sum - kvNum++ - bytes += (len(pair.Key) + len(pair.Val)) - } - - c.bytes += uint64(bytes) - c.kvs += uint64(kvNum) - c.checksum ^= checksum -} - -// Add other checksum. -func (c *Checksum) Add(other *Checksum) { - c.bytes += other.bytes - c.kvs += other.kvs - c.checksum ^= other.checksum -} - -// Sum returns the checksum. -func (c *Checksum) Sum() uint64 { - return c.checksum -} - -// SumSize returns the bytes. -func (c *Checksum) SumSize() uint64 { - return c.bytes -} - -// SumKVS returns the kv count. -func (c *Checksum) SumKVS() uint64 { - return c.kvs -} - -// MarshalLogObject implements the zapcore.ObjectMarshaler interface. -func (c *Checksum) MarshalLogObject(encoder zapcore.ObjectEncoder) error { - encoder.AddUint64("cksum", c.checksum) - encoder.AddUint64("size", c.bytes) - encoder.AddUint64("kvs", c.kvs) - return nil -} - -// MarshalJSON implements the json.Marshaler interface. -func (c Checksum) MarshalJSON() ([]byte, error) { - result := fmt.Sprintf(`{"checksum":%d,"size":%d,"kvs":%d}`, c.checksum, c.bytes, c.kvs) - return []byte(result), nil -} diff --git a/br/pkg/kv/checksum_test.go b/br/pkg/kv/checksum_test.go deleted file mode 100644 index 8b594f23f698c..0000000000000 --- a/br/pkg/kv/checksum_test.go +++ /dev/null @@ -1,80 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv_test - -import ( - "encoding/json" - "testing" - - "github.com/pingcap/tidb/br/pkg/kv" - "github.com/stretchr/testify/require" -) - -func uint64NotEqual(a uint64, b uint64) bool { return a != b } - -func TestChecksum(t *testing.T) { - checksum := kv.NewKVChecksum(0) - require.Equal(t, uint64(0), checksum.Sum()) - - // checksum on nothing - checksum.Update([]kv.Pair{}) - require.Equal(t, uint64(0), checksum.Sum()) - - checksum.Update(nil) - require.Equal(t, uint64(0), checksum.Sum()) - - // checksum on real data - expectChecksum := uint64(4850203904608948940) - - kvs := []kv.Pair{ - { - Key: []byte("Cop"), - Val: []byte("PingCAP"), - }, - { - Key: []byte("Introduction"), - Val: []byte("Inspired by Google Spanner/F1, PingCAP develops TiDB."), - }, - } - - checksum.Update(kvs) - - var kvBytes uint64 - for _, kv := range kvs { - kvBytes += uint64(len(kv.Key) + len(kv.Val)) - } - require.Equal(t, kvBytes, checksum.SumSize()) - require.Equal(t, uint64(len(kvs)), checksum.SumKVS()) - require.Equal(t, expectChecksum, checksum.Sum()) - - // recompute on same key-value - checksum.Update(kvs) - require.Equal(t, kvBytes<<1, checksum.SumSize()) - require.Equal(t, uint64(len(kvs))<<1, checksum.SumKVS()) - require.True(t, uint64NotEqual(checksum.Sum(), expectChecksum)) -} - -func TestChecksumJSON(t *testing.T) { - testStruct := &struct { - Checksum kv.Checksum - }{ - Checksum: kv.MakeKVChecksum(123, 456, 7890), - } - - res, err := json.Marshal(testStruct) - - require.NoError(t, err) - require.Equal(t, []byte(`{"Checksum":{"checksum":7890,"size":123,"kvs":456}}`), res) -} diff --git a/br/pkg/kv/kv.go b/br/pkg/kv/kv.go deleted file mode 100644 index 02c9457fb7fbb..0000000000000 --- a/br/pkg/kv/kv.go +++ /dev/null @@ -1,505 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "bytes" - "context" - "fmt" - "math" - "sort" - - "github.com/pingcap/errors" - sst "github.com/pingcap/kvproto/pkg/import_sstpb" - "github.com/pingcap/log" - "github.com/pingcap/tidb/br/pkg/logutil" - "github.com/pingcap/tidb/br/pkg/redact" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/types" - "go.uber.org/zap" -) - -var extraHandleColumnInfo = model.NewExtraHandleColInfo() - -// Iter abstract iterator method for Ingester. -type Iter interface { - // Seek seek to specify position. - // if key not found, seeks next key position in iter. - Seek(key []byte) bool - // Error return current error on this iter. - Error() error - // First moves this iter to the first key. - First() bool - // Last moves this iter to the last key. - Last() bool - // Valid check this iter reach the end. - Valid() bool - // Next moves this iter forward. - Next() bool - // Key represents current position pair's key. - Key() []byte - // Value represents current position pair's Value. - Value() []byte - // Close close this iter. - Close() error - // OpType represents operations of pair. currently we have two types. - // 1. Put - // 2. Delete - OpType() sst.Pair_OP -} - -// IterProducer produces iterator with given range. -type IterProducer interface { - // Produce produces iterator with given range [start, end). - Produce(start []byte, end []byte) Iter -} - -// SimpleKVIterProducer represents kv iter producer. -type SimpleKVIterProducer struct { - pairs Pairs -} - -// NewSimpleKVIterProducer creates SimpleKVIterProducer. -func NewSimpleKVIterProducer(pairs Pairs) IterProducer { - return &SimpleKVIterProducer{ - pairs: pairs, - } -} - -// Produce implements Iter.Producer.Produce. -func (p *SimpleKVIterProducer) Produce(start []byte, end []byte) Iter { - startIndex := sort.Search(len(p.pairs), func(i int) bool { - return bytes.Compare(start, p.pairs[i].Key) < 1 - }) - endIndex := sort.Search(len(p.pairs), func(i int) bool { - return bytes.Compare(end, p.pairs[i].Key) < 1 - }) - if startIndex >= endIndex { - log.Warn("produce failed due to start key is large than end key", - zap.Binary("start", start), zap.Binary("end", end)) - return nil - } - return newSimpleKVIter(p.pairs[startIndex:endIndex]) -} - -// SimpleKVIter represents simple pair iterator. -// which is used for log restore. -type SimpleKVIter struct { - index int - pairs Pairs -} - -// newSimpleKVIter creates SimpleKVIter. -func newSimpleKVIter(pairs Pairs) Iter { - return &SimpleKVIter{ - index: -1, - pairs: pairs, - } -} - -// Seek implements Iter.Seek. -func (s *SimpleKVIter) Seek(key []byte) bool { - s.index = sort.Search(len(s.pairs), func(i int) bool { - return bytes.Compare(key, s.pairs[i].Key) < 1 - }) - return s.index < len(s.pairs) -} - -// Error implements Iter.Error. -func (s *SimpleKVIter) Error() error { - return nil -} - -// First implements Iter.First. -func (s *SimpleKVIter) First() bool { - if len(s.pairs) == 0 { - return false - } - s.index = 0 - return true -} - -// Last implements Iter.Last. -func (s *SimpleKVIter) Last() bool { - if len(s.pairs) == 0 { - return false - } - s.index = len(s.pairs) - 1 - return true -} - -// Valid implements Iter.Valid. -func (s *SimpleKVIter) Valid() bool { - return s.index >= 0 && s.index < len(s.pairs) -} - -// Next implements Iter.Next. -func (s *SimpleKVIter) Next() bool { - s.index++ - return s.index < len(s.pairs) -} - -// Key implements Iter.Key. -func (s *SimpleKVIter) Key() []byte { - if s.index >= 0 && s.index < len(s.pairs) { - return s.pairs[s.index].Key - } - return nil -} - -// Value implements Iter.Value. -func (s *SimpleKVIter) Value() []byte { - if s.index >= 0 && s.index < len(s.pairs) { - return s.pairs[s.index].Val - } - return nil -} - -// Close implements Iter.Close. -func (s *SimpleKVIter) Close() error { - return nil -} - -// OpType implements Iter.KeyIsDelete. -func (s *SimpleKVIter) OpType() sst.Pair_OP { - if s.Valid() && s.pairs[s.index].IsDelete { - return sst.Pair_Delete - } - return sst.Pair_Put -} - -// Encoder encodes a row of SQL values into some opaque type which can be -// consumed by OpenEngine.WriteEncoded. -type Encoder interface { - // Close the encoder. - Close() - - // AddRecord encode encodes a row of SQL values into a backend-friendly format. - AddRecord( - row []types.Datum, - rowID int64, - columnPermutation []int, - ) (Row, int, error) - - // RemoveRecord encode encodes a row of SQL delete values into a backend-friendly format. - RemoveRecord( - row []types.Datum, - rowID int64, - columnPermutation []int, - ) (Row, int, error) -} - -// Row represents a single encoded row. -type Row interface { - // ClassifyAndAppend separates the data-like and index-like parts of the - // encoded row, and appends these parts into the existing buffers and - // checksums. - ClassifyAndAppend( - data *Pairs, - dataChecksum *Checksum, - indices *Pairs, - indexChecksum *Checksum, - ) -} - -type tableKVEncoder struct { - tbl table.Table - se *session - recordCache []types.Datum -} - -// NewTableKVEncoder creates the Encoder. -func NewTableKVEncoder(tbl table.Table, options *SessionOptions) Encoder { - se := newSession(options) - // Set CommonAddRecordCtx to session to reuse the slices and BufStore in AddRecord - recordCtx := tables.NewCommonAddRecordCtx(len(tbl.Cols())) - tables.SetAddRecordCtx(se, recordCtx) - return &tableKVEncoder{ - tbl: tbl, - se: se, - } -} - -var kindStr = [...]string{ - types.KindNull: "null", - types.KindInt64: "int64", - types.KindUint64: "uint64", - types.KindFloat32: "float32", - types.KindFloat64: "float64", - types.KindString: "string", - types.KindBytes: "bytes", - types.KindBinaryLiteral: "binary", - types.KindMysqlDecimal: "decimal", - types.KindMysqlDuration: "duration", - types.KindMysqlEnum: "enum", - types.KindMysqlBit: "bit", - types.KindMysqlSet: "set", - types.KindMysqlTime: "time", - types.KindInterface: "interface", - types.KindMinNotNull: "min", - types.KindMaxValue: "max", - types.KindRaw: "raw", - types.KindMysqlJSON: "json", -} - -// MarshalLogArray implements the zapcore.ArrayMarshaler interface. -func zapRow(key string, row []types.Datum) zap.Field { - return logutil.AbbreviatedArray(key, row, func(input interface{}) []string { - row := input.([]types.Datum) - vals := make([]string, 0, len(row)) - for _, datum := range row { - kind := datum.Kind() - var str string - var err error - switch kind { - case types.KindNull: - str = "NULL" - case types.KindMinNotNull: - str = "-inf" - case types.KindMaxValue: - str = "+inf" - default: - str, err = datum.ToString() - if err != nil { - vals = append(vals, err.Error()) - continue - } - } - vals = append(vals, - fmt.Sprintf("kind: %s, val: %s", kindStr[kind], redact.String(str))) - } - return vals - }) -} - -// Pairs represents the slice of Pair. -type Pairs []Pair - -// Close ... -func (kvcodec *tableKVEncoder) Close() { -} - -// AddRecord encode a row of data into KV pairs. -// -// See comments in `(*TableRestore).initializeColumns` for the meaning of the -// `columnPermutation` parameter. -func (kvcodec *tableKVEncoder) AddRecord( - row []types.Datum, - rowID int64, - columnPermutation []int, -) (Row, int, error) { - cols := kvcodec.tbl.Cols() - - var value types.Datum - var err error - - record := kvcodec.recordCache - if record == nil { - record = make([]types.Datum, 0, len(cols)+1) - } - - isAutoRandom := false - if kvcodec.tbl.Meta().PKIsHandle && kvcodec.tbl.Meta().ContainsAutoRandomBits() { - isAutoRandom = true - } - - for i, col := range cols { - j := columnPermutation[i] - isAutoIncCol := mysql.HasAutoIncrementFlag(col.Flag) - isPk := mysql.HasPriKeyFlag(col.Flag) - switch { - case j >= 0 && j < len(row): - value, err = table.CastValue(kvcodec.se, row[j], col.ToInfo(), false, false) - if err == nil { - err = col.HandleBadNull(&value, kvcodec.se.vars.StmtCtx) - } - case isAutoIncCol: - // we still need a conversion, e.g. to catch overflow with a TINYINT column. - value, err = table.CastValue(kvcodec.se, types.NewIntDatum(rowID), col.ToInfo(), false, false) - default: - value, err = table.GetColDefaultValue(kvcodec.se, col.ToInfo()) - } - if err != nil { - return nil, 0, errors.Trace(err) - } - - record = append(record, value) - - if isAutoRandom && isPk { - typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[col.Tp] * 8) - incrementalBits := typeBitsLength - kvcodec.tbl.Meta().AutoRandomBits - hasSignBit := !mysql.HasUnsignedFlag(col.Flag) - if hasSignBit { - incrementalBits-- - } - alloc := kvcodec.tbl.Allocators(kvcodec.se).Get(autoid.AutoRandomType) - _ = alloc.Rebase(context.Background(), value.GetInt64()&((1<= 0 && j < len(row) { - value, err = table.CastValue(kvcodec.se, row[j], extraHandleColumnInfo, false, false) - } else { - value, err = types.NewIntDatum(rowID), nil - } - if err != nil { - return nil, 0, errors.Trace(err) - } - record = append(record, value) - alloc := kvcodec.tbl.Allocators(kvcodec.se).Get(autoid.RowIDAllocType) - _ = alloc.Rebase(context.Background(), value.GetInt64(), false) - } - _, err = kvcodec.tbl.AddRecord(kvcodec.se, record) - if err != nil { - log.Error("kv add Record failed", - zapRow("originalRow", row), - zapRow("convertedRow", record), - zap.Error(err), - ) - return nil, 0, errors.Trace(err) - } - - pairs, size := kvcodec.se.takeKvPairs() - kvcodec.recordCache = record[:0] - return Pairs(pairs), size, nil -} - -// get record value for auto-increment field -// -// See: https://github.com/pingcap/tidb/blob/47f0f15b14ed54fc2222f3e304e29df7b05e6805/executor/insert_common.go#L781-L852 -// TODO: merge this with pkg/lightning/backend/kv/sql2kv.go -func getAutoRecordID(d types.Datum, target *types.FieldType) int64 { - switch target.Tp { - case mysql.TypeFloat, mysql.TypeDouble: - return int64(math.Round(d.GetFloat64())) - case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: - return d.GetInt64() - default: - panic(fmt.Sprintf("unsupported auto-increment field type '%d'", target.Tp)) - } -} - -// RemoveRecord encode a row of data into KV pairs. -func (kvcodec *tableKVEncoder) RemoveRecord( - row []types.Datum, - rowID int64, - columnPermutation []int, -) (Row, int, error) { - cols := kvcodec.tbl.Cols() - - var value types.Datum - var err error - - record := kvcodec.recordCache - if record == nil { - record = make([]types.Datum, 0, len(cols)+1) - } - - for i, col := range cols { - j := columnPermutation[i] - isAutoIncCol := mysql.HasAutoIncrementFlag(col.Flag) - switch { - case j >= 0 && j < len(row): - value, err = table.CastValue(kvcodec.se, row[j], col.ToInfo(), false, false) - if err == nil { - err = col.HandleBadNull(&value, kvcodec.se.vars.StmtCtx) - } - case isAutoIncCol: - // we still need a conversion, e.g. to catch overflow with a TINYINT column. - value, err = table.CastValue(kvcodec.se, types.NewIntDatum(rowID), col.ToInfo(), false, false) - default: - value, err = table.GetColDefaultValue(kvcodec.se, col.ToInfo()) - } - if err != nil { - return nil, 0, errors.Trace(err) - } - record = append(record, value) - } - err = kvcodec.tbl.RemoveRecord(kvcodec.se, kv.IntHandle(rowID), record) - if err != nil { - log.Error("kv remove record failed", - zapRow("originalRow", row), - zapRow("convertedRow", record), - zap.Error(err), - ) - return nil, 0, errors.Trace(err) - } - - pairs, size := kvcodec.se.takeKvPairs() - kvcodec.recordCache = record[:0] - return Pairs(pairs), size, nil -} - -// ClassifyAndAppend split Pairs to data rows and index rows. -func (kvs Pairs) ClassifyAndAppend( - data *Pairs, - dataChecksum *Checksum, - indices *Pairs, - indexChecksum *Checksum, -) { - dataKVs := *data - indexKVs := *indices - - for _, kv := range kvs { - if kv.Key[tablecodec.TableSplitKeyLen+1] == 'r' { - dataKVs = append(dataKVs, kv) - dataChecksum.UpdateOne(kv) - } else { - indexKVs = append(indexKVs, kv) - indexChecksum.UpdateOne(kv) - } - } - - *data = dataKVs - *indices = indexKVs -} - -// Clear resets the Pairs. -func (kvs Pairs) Clear() Pairs { - return kvs[:0] -} - -// NextKey return the smallest []byte that is bigger than current bytes. -// special case when key is empty, empty bytes means infinity in our context, so directly return itself. -func NextKey(key []byte) []byte { - if len(key) == 0 { - return []byte{} - } - - // in tikv <= 4.x, tikv will truncate the row key, so we should fetch the next valid row key - // See: https://github.com/tikv/tikv/blob/f7f22f70e1585d7ca38a59ea30e774949160c3e8/components/raftstore/src/coprocessor/split_observer.rs#L36-L41 - if tablecodec.IsRecordKey(key) { - tableID, handle, _ := tablecodec.DecodeRecordKey(key) - return tablecodec.EncodeRowKeyWithHandle(tableID, handle.Next()) - } - - // if key is an index, directly append a 0x00 to the key. - res := make([]byte, 0, len(key)+1) - res = append(res, key...) - res = append(res, 0) - return res -} diff --git a/br/pkg/kv/kv_test.go b/br/pkg/kv/kv_test.go deleted file mode 100644 index ddf32247e03b2..0000000000000 --- a/br/pkg/kv/kv_test.go +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "bytes" - "strings" - "testing" - - "github.com/pingcap/tidb/types" - "github.com/stretchr/testify/require" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" -) - -func TestMarshal(t *testing.T) { - dats := make([]types.Datum, 4) - dats[0].SetInt64(1) - dats[1].SetNull() - dats[2] = types.MaxValueDatum() - dats[3] = types.MinNotNullDatum() - - encoder := zapcore.NewConsoleEncoder(zapcore.EncoderConfig{}) - out, err := encoder.EncodeEntry(zapcore.Entry{}, []zap.Field{zapRow("row", dats)}) - require.NoError(t, err) - require.Equal(t, - `{"row": ["kind: int64, val: 1", "kind: null, val: NULL", "kind: max, val: +inf", "kind: min, val: -inf"]}`, - strings.TrimRight(out.String(), "\n")) -} - -func TestSimplePairIter(t *testing.T) { - pairs := []Pair{ - {Key: []byte("1"), Val: []byte("a")}, - {Key: []byte("2"), Val: []byte("b")}, - {Key: []byte("3"), Val: []byte("c")}, - {Key: []byte("5"), Val: []byte("d")}, - } - expectCount := 4 - iter := newSimpleKVIter(pairs) - count := 0 - for iter.Next() { - count++ - } - require.Equal(t, expectCount, count) - - require.True(t, iter.First()) - require.True(t, iter.Last()) - - require.True(t, iter.Seek([]byte("1"))) - require.True(t, bytes.Equal(iter.Key(), []byte("1"))) - require.True(t, bytes.Equal(iter.Value(), []byte("a"))) - require.True(t, iter.Valid()) - - require.True(t, iter.Seek([]byte("2"))) - require.True(t, bytes.Equal(iter.Key(), []byte("2"))) - require.True(t, bytes.Equal(iter.Value(), []byte("b"))) - require.True(t, iter.Valid()) - - require.True(t, iter.Seek([]byte("3"))) - require.True(t, bytes.Equal(iter.Key(), []byte("3"))) - require.True(t, bytes.Equal(iter.Value(), []byte("c"))) - require.True(t, iter.Valid()) - - // 4 not exists, so seek position will move to 5. - require.True(t, iter.Seek([]byte("4"))) - require.True(t, bytes.Equal(iter.Key(), []byte("5"))) - require.True(t, bytes.Equal(iter.Value(), []byte("d"))) - require.True(t, iter.Valid()) - - // 6 not exists, so seek position will not valid. - require.False(t, iter.Seek([]byte("6"))) - require.False(t, iter.Valid()) -} diff --git a/br/pkg/kv/session.go b/br/pkg/kv/session.go deleted file mode 100644 index 9b1f453f0308b..0000000000000 --- a/br/pkg/kv/session.go +++ /dev/null @@ -1,259 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -import ( - "context" - "fmt" - "strconv" - - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/topsql/stmtstats" -) - -// Pair is a pair of key and value. -type Pair struct { - // Key is the key of the KV pair - Key []byte - // Val is the value of the KV pair - Val []byte - // IsDelete represents whether we should remove this KV pair. - IsDelete bool -} - -// invalidIterator is a trimmed down Iterator type which is invalid. -type invalidIterator struct { - kv.Iterator -} - -// TableHasAutoRowID return whether table has auto generated row id. -func TableHasAutoRowID(info *model.TableInfo) bool { - return !info.PKIsHandle && !info.IsCommonHandle -} - -// Valid implements the kv.Iterator interface. -func (*invalidIterator) Valid() bool { - return false -} - -// Close implements the kv.Iterator interface. -func (*invalidIterator) Close() { -} - -type kvMemBuf struct { - kv.MemBuffer - kvPairs []Pair - size int -} - -func (mb *kvMemBuf) Set(k kv.Key, v []byte) error { - mb.kvPairs = append(mb.kvPairs, Pair{ - Key: k.Clone(), - Val: append([]byte{}, v...), - }) - mb.size += len(k) + len(v) - return nil -} - -func (mb *kvMemBuf) SetWithFlags(k kv.Key, v []byte, ops ...kv.FlagsOp) error { - return mb.Set(k, v) -} - -func (mb *kvMemBuf) Delete(k kv.Key) error { - mb.kvPairs = append(mb.kvPairs, Pair{ - Key: k.Clone(), - Val: []byte{}, - IsDelete: true, - }) - mb.size += len(k) - return nil -} - -func (mb *kvMemBuf) DeleteWithFlags(k kv.Key, ops ...kv.FlagsOp) error { - return mb.Delete(k) -} - -// Release publish all modifications in the latest staging buffer to upper level. -func (mb *kvMemBuf) Release(h kv.StagingHandle) { -} - -func (mb *kvMemBuf) Staging() kv.StagingHandle { - return 0 -} - -// Cleanup cleanup the resources referenced by the StagingHandle. -// If the changes are not published by `Release`, they will be discarded. -func (mb *kvMemBuf) Cleanup(h kv.StagingHandle) {} - -// Size returns sum of keys and values length. -func (mb *kvMemBuf) Size() int { - return mb.size -} - -// Len returns the number of entries in the DB. -func (t *transaction) Len() int { - return t.GetMemBuffer().Len() -} - -type kvUnionStore struct { - kvMemBuf -} - -func (s *kvUnionStore) GetMemBuffer() kv.MemBuffer { - return &s.kvMemBuf -} - -func (s *kvUnionStore) GetIndexName(tableID, indexID int64) string { - panic("Unsupported Operation") -} - -func (s *kvUnionStore) CacheIndexName(tableID, indexID int64, name string) { -} - -func (s *kvUnionStore) CacheTableInfo(id int64, info *model.TableInfo) { -} - -// transaction is a trimmed down Transaction type which only supports adding a -// new KV pair. -type transaction struct { - kv.Transaction - kvUnionStore -} - -func (t *transaction) GetMemBuffer() kv.MemBuffer { - return &t.kvUnionStore.kvMemBuf -} - -func (t *transaction) Discard() { - // do nothing -} - -func (t *transaction) Flush() (int, error) { - // do nothing - return 0, nil -} - -// Reset implements the kv.MemBuffer interface. -func (t *transaction) Reset() {} - -// Get implements the kv.Retriever interface. -func (t *transaction) Get(ctx context.Context, key kv.Key) ([]byte, error) { - return nil, kv.ErrNotExist -} - -// Iter implements the kv.Retriever interface. -func (t *transaction) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { - return &invalidIterator{}, nil -} - -// Set implements the kv.Mutator interface. -func (t *transaction) Set(k kv.Key, v []byte) error { - return t.kvMemBuf.Set(k, v) -} - -// Delete implements the kv.Mutator interface. -func (t *transaction) Delete(k kv.Key) error { - return t.kvMemBuf.Delete(k) -} - -// GetTableInfo implements the kv.Transaction interface. -func (t *transaction) GetTableInfo(id int64) *model.TableInfo { - return nil -} - -// CacheTableInfo implements the kv.Transaction interface. -func (t *transaction) CacheTableInfo(id int64, info *model.TableInfo) { -} - -// session is a trimmed down Session type which only wraps our own trimmed-down -// transaction type and provides the session variables to the TiDB library -// optimized for Lightning. -type session struct { - sessionctx.Context - txn transaction - vars *variable.SessionVars - // currently, we only set `CommonAddRecordCtx` - values map[fmt.Stringer]interface{} -} - -// SessionOptions is the initial configuration of the session. -type SessionOptions struct { - SQLMode mysql.SQLMode - Timestamp int64 - RowFormatVersion string -} - -func newSession(options *SessionOptions) *session { - sqlMode := options.SQLMode - vars := variable.NewSessionVars() - vars.SkipUTF8Check = true - vars.StmtCtx.InInsertStmt = true - vars.StmtCtx.BatchCheck = true - vars.StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode() - vars.StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode() - vars.StmtCtx.OverflowAsWarning = !sqlMode.HasStrictMode() - vars.StmtCtx.AllowInvalidDate = sqlMode.HasAllowInvalidDatesMode() - vars.StmtCtx.IgnoreZeroInDate = !sqlMode.HasStrictMode() || sqlMode.HasAllowInvalidDatesMode() - vars.StmtCtx.TimeZone = vars.Location() - _ = vars.SetSystemVar("timestamp", strconv.FormatInt(options.Timestamp, 10)) - _ = vars.SetSystemVar(variable.TiDBRowFormatVersion, options.RowFormatVersion) - vars.TxnCtx = nil - - s := &session{ - vars: vars, - values: make(map[fmt.Stringer]interface{}, 1), - } - return s -} - -func (se *session) takeKvPairs() ([]Pair, int) { - pairs := se.txn.kvMemBuf.kvPairs - size := se.txn.kvMemBuf.Size() - se.txn.kvMemBuf.kvPairs = make([]Pair, 0, len(pairs)) - se.txn.kvMemBuf.size = 0 - return pairs, size -} - -// Txn implements the sessionctx.Context interface. -func (se *session) Txn(active bool) (kv.Transaction, error) { - return &se.txn, nil -} - -// GetSessionVars implements the sessionctx.Context interface. -func (se *session) GetSessionVars() *variable.SessionVars { - return se.vars -} - -// SetValue saves a value associated with this context for key. -func (se *session) SetValue(key fmt.Stringer, value interface{}) { - se.values[key] = value -} - -// Value returns the value associated with this context for key. -func (se *session) Value(key fmt.Stringer) interface{} { - return se.values[key] -} - -// StmtAddDirtyTableOP implements the sessionctx.Context interface. -func (se *session) StmtAddDirtyTableOP(op int, physicalID int64, handle kv.Handle) {} - -// GetStmtStats implements the sessionctx.Context interface. -func (se *session) GetStmtStats() *stmtstats.StatementStats { - return nil -} diff --git a/br/pkg/lightning/backend/kv/session.go b/br/pkg/lightning/backend/kv/session.go index fb49514bcd879..ca0cb2db4529c 100644 --- a/br/pkg/lightning/backend/kv/session.go +++ b/br/pkg/lightning/backend/kv/session.go @@ -219,6 +219,11 @@ func (t *transaction) GetTableInfo(id int64) *model.TableInfo { func (t *transaction) CacheTableInfo(id int64, info *model.TableInfo) { } +// SetAssertion implements the kv.Transaction interface. +func (t *transaction) SetAssertion(key []byte, assertion ...kv.FlagsOp) error { + return nil +} + // session is a trimmed down Session type which only wraps our own trimmed-down // transaction type and provides the session variables to the TiDB library // optimized for Lightning. diff --git a/br/pkg/lightning/backend/kv/sql2kv_test.go b/br/pkg/lightning/backend/kv/sql2kv_test.go index 03c121591862b..56d9c6b32b249 100644 --- a/br/pkg/lightning/backend/kv/sql2kv_test.go +++ b/br/pkg/lightning/backend/kv/sql2kv_test.go @@ -596,7 +596,7 @@ func BenchmarkSQL2KV(b *testing.B) { for i := 0; i < b.N; i++ { rows, err := s.encoder.Encode(s.logger, s.row, 1, s.colPerm, "", 0) require.NoError(b, err) - len := reflect.ValueOf(rows).Elem().Field(0).Len() - require.Equal(b, len, 2) + l := reflect.ValueOf(rows).Elem().Field(0).Len() + require.Equal(b, l, 2) } } diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index 18dade43eb60d..983ae33fcfd68 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" - pkgkv "github.com/pingcap/tidb/br/pkg/kv" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/errormanager" @@ -67,12 +66,12 @@ type pendingIndexHandles struct { // makePendingIndexHandlesWithCapacity makes the pendingIndexHandles struct-of-arrays with the given // capacity for every internal array. -func makePendingIndexHandlesWithCapacity(cap int) pendingIndexHandles { +func makePendingIndexHandlesWithCapacity(capacity int) pendingIndexHandles { return pendingIndexHandles{ - dataConflictInfos: make([]errormanager.DataConflictInfo, 0, cap), - indexNames: make([]string, 0, cap), - handles: make([]tidbkv.Handle, 0, cap), - rawHandles: make([][]byte, 0, cap), + dataConflictInfos: make([]errormanager.DataConflictInfo, 0, capacity), + indexNames: make([]string, 0, capacity), + handles: make([]tidbkv.Handle, 0, capacity), + rawHandles: make([][]byte, 0, capacity), } } @@ -248,7 +247,7 @@ type DupKVStream interface { // It collects duplicate key-value pairs from a pebble.DB. //goland:noinspection GoNameStartsWithPackageName type LocalDupKVStream struct { - iter pkgkv.Iter + iter Iter } // NewLocalDupKVStream creates a new LocalDupKVStream with the given duplicate db and key range. diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go index 29fac0e200e48..f14a9a06c8e0a 100644 --- a/br/pkg/lightning/backend/local/engine.go +++ b/br/pkg/lightning/backend/local/engine.go @@ -33,7 +33,6 @@ import ( "github.com/google/btree" "github.com/google/uuid" "github.com/pingcap/errors" - pkgkv "github.com/pingcap/tidb/br/pkg/kv" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" @@ -945,7 +944,7 @@ func (e *Engine) unfinishedRanges(ranges []Range) []Range { return filterOverlapRange(ranges, e.finishedRanges.ranges) } -func (e *Engine) newKVIter(ctx context.Context, opts *pebble.IterOptions) pkgkv.Iter { +func (e *Engine) newKVIter(ctx context.Context, opts *pebble.IterOptions) Iter { if bytes.Compare(opts.LowerBound, normalIterStartKey) < 0 { newOpts := *opts newOpts.LowerBound = normalIterStartKey diff --git a/br/pkg/lightning/backend/local/iterator.go b/br/pkg/lightning/backend/local/iterator.go index 3652b0e0243fa..16c9b647e4f24 100644 --- a/br/pkg/lightning/backend/local/iterator.go +++ b/br/pkg/lightning/backend/local/iterator.go @@ -21,12 +21,38 @@ import ( "github.com/cockroachdb/pebble" sst "github.com/pingcap/kvproto/pkg/import_sstpb" - "github.com/pingcap/tidb/br/pkg/kv" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/logutil" "go.uber.org/multierr" ) +// Iter abstract iterator method for Ingester. +type Iter interface { + // Seek seek to specify position. + // if key not found, seeks next key position in iter. + Seek(key []byte) bool + // Error return current error on this iter. + Error() error + // First moves this iter to the first key. + First() bool + // Last moves this iter to the last key. + Last() bool + // Valid check this iter reach the end. + Valid() bool + // Next moves this iter forward. + Next() bool + // Key represents current position pair's key. + Key() []byte + // Value represents current position pair's Value. + Value() []byte + // Close close this iter. + Close() error + // OpType represents operations of pair. currently we have two types. + // 1. Put + // 2. Delete + OpType() sst.Pair_OP +} + type pebbleIter struct { *pebble.Iterator } @@ -39,7 +65,7 @@ func (p pebbleIter) OpType() sst.Pair_OP { return sst.Pair_Put } -var _ kv.Iter = pebbleIter{} +var _ Iter = pebbleIter{} const maxDuplicateBatchSize = 4 << 20 @@ -163,7 +189,7 @@ func (d *dupDetectIter) OpType() sst.Pair_OP { return sst.Pair_Put } -var _ kv.Iter = &dupDetectIter{} +var _ Iter = &dupDetectIter{} func newDupDetectIter(ctx context.Context, db *pebble.DB, keyAdapter KeyAdapter, opts *pebble.IterOptions, dupDB *pebble.DB, logger log.Logger) *dupDetectIter { @@ -250,7 +276,7 @@ func (d *dupDBIter) OpType() sst.Pair_OP { return sst.Pair_Put } -var _ kv.Iter = &dupDBIter{} +var _ Iter = &dupDBIter{} func newDupDBIter(dupDB *pebble.DB, keyAdapter KeyAdapter, opts *pebble.IterOptions) *dupDBIter { newOpts := &pebble.IterOptions{TableFilter: opts.TableFilter} diff --git a/br/pkg/lightning/backend/local/iterator_test.go b/br/pkg/lightning/backend/local/iterator_test.go index da8ada357ac6b..3abb6fbc3b06c 100644 --- a/br/pkg/lightning/backend/local/iterator_test.go +++ b/br/pkg/lightning/backend/local/iterator_test.go @@ -24,7 +24,6 @@ import ( "time" "github.com/cockroachdb/pebble" - "github.com/pingcap/tidb/br/pkg/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/stretchr/testify/require" @@ -122,7 +121,7 @@ func TestDupDetectIterator(t *testing.T) { dupDB, err := pebble.Open(filepath.Join(storeDir, "duplicates"), &pebble.Options{}) require.NoError(t, err) - var iter kv.Iter + var iter Iter iter = newDupDetectIter(context.Background(), db, keyAdapter, &pebble.IterOptions{}, dupDB, log.L()) sort.Slice(pairs, func(i, j int) bool { key1 := keyAdapter.Encode(nil, pairs[i].Key, pairs[i].RowID) diff --git a/br/pkg/lightning/backend/local/key_adapter_test.go b/br/pkg/lightning/backend/local/key_adapter_test.go index 8e51586f54a86..4b9abe1c25c3f 100644 --- a/br/pkg/lightning/backend/local/key_adapter_test.go +++ b/br/pkg/lightning/backend/local/key_adapter_test.go @@ -34,7 +34,7 @@ func randBytes(n int) []byte { func TestNoopKeyAdapter(t *testing.T) { keyAdapter := noopKeyAdapter{} key := randBytes(32) - require.Equal(t, len(key), keyAdapter.EncodedLen(key)) + require.Len(t, key, keyAdapter.EncodedLen(key)) encodedKey := keyAdapter.Encode(nil, key, 0) require.Equal(t, key, encodedKey) diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index 54a75539c0510..dd52b71493507 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -32,12 +32,11 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" - "github.com/tikv/pd/server/core" - "github.com/tikv/pd/server/schedule/placement" "go.uber.org/atomic" ) @@ -51,7 +50,7 @@ type testClient struct { mu sync.RWMutex stores map[uint64]*metapb.Store regions map[uint64]*restore.RegionInfo - regionsInfo *core.RegionsInfo // For now it's only used in ScanRegions + regionsInfo *pdtypes.RegionTree // For now it's only used in ScanRegions nextRegionID uint64 splitCount atomic.Int32 hook clientHook @@ -63,9 +62,9 @@ func newTestClient( nextRegionID uint64, hook clientHook, ) *testClient { - regionsInfo := core.NewRegionsInfo() + regionsInfo := &pdtypes.RegionTree{} for _, regionInfo := range regions { - regionsInfo.SetRegion(core.NewRegionInfo(regionInfo.Region, regionInfo.Leader)) + regionsInfo.SetRegion(pdtypes.NewRegionInfo(regionInfo.Region, regionInfo.Leader)) } return &testClient{ stores: stores, @@ -150,12 +149,12 @@ func (c *testClient) SplitRegion( }, } c.regions[c.nextRegionID] = newRegion - c.regionsInfo.SetRegion(core.NewRegionInfo(newRegion.Region, newRegion.Leader)) + c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(newRegion.Region, newRegion.Leader)) c.nextRegionID++ target.Region.StartKey = splitKey target.Region.RegionEpoch.ConfVer++ c.regions[target.Region.Id] = target - c.regionsInfo.SetRegion(core.NewRegionInfo(target.Region, target.Leader)) + c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(target.Region, target.Leader)) return newRegion, nil } @@ -211,7 +210,7 @@ func (c *testClient) BatchSplitRegionsWithOrigin( }, } c.regions[c.nextRegionID] = newRegion - c.regionsInfo.SetRegion(core.NewRegionInfo(newRegion.Region, newRegion.Leader)) + c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(newRegion.Region, newRegion.Leader)) c.nextRegionID++ startKey = key newRegions = append(newRegions, newRegion) @@ -219,7 +218,7 @@ func (c *testClient) BatchSplitRegionsWithOrigin( if !bytes.Equal(target.Region.StartKey, startKey) { target.Region.StartKey = startKey c.regions[target.Region.Id] = target - c.regionsInfo.SetRegion(core.NewRegionInfo(target.Region, target.Leader)) + c.regionsInfo.SetRegion(pdtypes.NewRegionInfo(target.Region, target.Leader)) } if len(newRegions) == 0 { @@ -260,8 +259,8 @@ func (c *testClient) ScanRegions(ctx context.Context, key, endKey []byte, limit regions := make([]*restore.RegionInfo, 0, len(infos)) for _, info := range infos { regions = append(regions, &restore.RegionInfo{ - Region: info.GetMeta(), - Leader: info.GetLeader(), + Region: info.Meta, + Leader: info.Leader, }) } @@ -272,11 +271,11 @@ func (c *testClient) ScanRegions(ctx context.Context, key, endKey []byte, limit return regions, err } -func (c *testClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r placement.Rule, err error) { +func (c *testClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r pdtypes.Rule, err error) { return } -func (c *testClient) SetPlacementRule(ctx context.Context, rule placement.Rule) error { +func (c *testClient) SetPlacementRule(ctx context.Context, rule pdtypes.Rule) error { return nil } diff --git a/br/pkg/lightning/checkpoints/checkpoints.go b/br/pkg/lightning/checkpoints/checkpoints.go index 47ced55db41c5..45530c5bee0c7 100644 --- a/br/pkg/lightning/checkpoints/checkpoints.go +++ b/br/pkg/lightning/checkpoints/checkpoints.go @@ -1302,7 +1302,6 @@ func (cpdb *MySQLCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[ // 2. engine status is earlier than CheckpointStatusImported, and // 3. chunk has been read - // nolint:gosec query := fmt.Sprintf(` SELECT DISTINCT t.table_name, c.engine_id FROM %s.%s t, %s.%s c, %s.%s e @@ -1316,7 +1315,7 @@ func (cpdb *MySQLCheckpointsDB) GetLocalStoringTables(ctx context.Context) (map[ err := common.Retry("get local storing tables", log.L(), func() error { targetTables = make(map[string][]int32) - rows, err := cpdb.db.QueryContext(ctx, query) + rows, err := cpdb.db.QueryContext(ctx, query) // #nosec G201 if err != nil { return errors.Trace(err) } @@ -1388,7 +1387,7 @@ func (cpdb *MySQLCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tabl colName = columnTableName aliasedColName = "t.table_name" } - // nolint:gosec + selectQuery := fmt.Sprintf(` SELECT t.table_name, @@ -1418,7 +1417,7 @@ func (cpdb *MySQLCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tabl err := s.Transact(ctx, "destroy error checkpoints", func(c context.Context, tx *sql.Tx) error { // Obtain the list of tables targetTables = nil - rows, e := tx.QueryContext(c, selectQuery, tableName) + rows, e := tx.QueryContext(c, selectQuery, tableName) // #nosec G201 if e != nil { return errors.Trace(e) } diff --git a/br/pkg/lightning/checkpoints/checkpoints_file_test.go b/br/pkg/lightning/checkpoints/checkpoints_file_test.go index bb3a374ad176d..947819fd77999 100644 --- a/br/pkg/lightning/checkpoints/checkpoints_file_test.go +++ b/br/pkg/lightning/checkpoints/checkpoints_file_test.go @@ -6,24 +6,14 @@ import ( "sort" "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/verification" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&cpFileSuite{}) - -type cpFileSuite struct { - cpdb *checkpoints.FileCheckpointsDB -} - func newTestConfig() *config.Config { cfg := config.NewConfig() cfg.Mydumper.SourceDir = "/data" @@ -36,12 +26,10 @@ func newTestConfig() *config.Config { return cfg } -func (s *cpFileSuite) SetUpTest(c *C) { - dir := c.MkDir() - s.cpdb = checkpoints.NewFileCheckpointsDB(filepath.Join(dir, "cp.pb")) - +func newFileCheckpointsDB(t *testing.T) (*checkpoints.FileCheckpointsDB, func()) { + dir := t.TempDir() + cpdb := checkpoints.NewFileCheckpointsDB(filepath.Join(dir, "cp.pb")) ctx := context.Background() - cpdb := s.cpdb // 2. initialize with checkpoint data. cfg := newTestConfig() @@ -60,7 +48,7 @@ func (s *cpFileSuite) SetUpTest(c *C) { }, }, }) - c.Assert(err, IsNil) + require.NoError(t, err) // 3. set some checkpoints @@ -90,7 +78,7 @@ func (s *cpFileSuite) SetUpTest(c *C) { Chunks: nil, }, }) - c.Assert(err, IsNil) + require.NoError(t, err) err = cpdb.InsertEngineCheckpoints(ctx, "`db2`.`t3`", map[int32]*checkpoints.EngineCheckpoint{ -1: { @@ -98,7 +86,7 @@ func (s *cpFileSuite) SetUpTest(c *C) { Chunks: nil, }, }) - c.Assert(err, IsNil) + require.NoError(t, err) // 4. update some checkpoints @@ -131,13 +119,13 @@ func (s *cpFileSuite) SetUpTest(c *C) { ccm.MergeInto(cpd) cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) + return cpdb, func() { + err := cpdb.Close() + require.NoError(t, err) + } } -func (s *cpFileSuite) TearDownTest(c *C) { - c.Assert(s.cpdb.Close(), IsNil) -} - -func (s *cpFileSuite) setInvalidStatus() { +func setInvalidStatus(cpdb *checkpoints.FileCheckpointsDB) { cpd := checkpoints.NewTableCheckpointDiff() scm := checkpoints.StatusCheckpointMerger{ EngineID: -1, @@ -146,20 +134,22 @@ func (s *cpFileSuite) setInvalidStatus() { scm.SetInvalid() scm.MergeInto(cpd) - s.cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{ + cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{ "`db1`.`t2`": cpd, "`db2`.`t3`": cpd, }) } -func (s *cpFileSuite) TestGet(c *C) { +func TestGet(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() // 5. get back the checkpoints - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(cp, DeepEquals, &checkpoints.TableCheckpoint{ + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.NoError(t, err) + expect := &checkpoints.TableCheckpoint{ Status: checkpoints.CheckpointStatusAllWritten, AllocBase: 132861, Checksum: verification.MakeKVChecksum(4492, 686, 486070148910), @@ -191,11 +181,12 @@ func (s *cpFileSuite) TestGet(c *C) { }}, }, }, - }) + } + require.Equal(t, expect, cp) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(err, IsNil) - c.Assert(cp, DeepEquals, &checkpoints.TableCheckpoint{ + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.NoError(t, err) + expect = &checkpoints.TableCheckpoint{ Status: checkpoints.CheckpointStatusLoaded, Engines: map[int32]*checkpoints.EngineCheckpoint{ -1: { @@ -203,86 +194,97 @@ func (s *cpFileSuite) TestGet(c *C) { Chunks: []*checkpoints.ChunkCheckpoint{}, }, }, - }) + } + require.Equal(t, expect, cp) - cp, err = s.cpdb.Get(ctx, "`db3`.`not-exists`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err = cpdb.Get(ctx, "`db3`.`not-exists`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) } -func (s *cpFileSuite) TestRemoveAllCheckpoints(c *C) { +func TestRemoveAllCheckpoints(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - err := s.cpdb.RemoveCheckpoint(ctx, "all") - c.Assert(err, IsNil) + err := cpdb.RemoveCheckpoint(ctx, "all") + require.NoError(t, err) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) } -func (s *cpFileSuite) TestRemoveOneCheckpoint(c *C) { +func TestRemoveOneCheckpoint(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - err := s.cpdb.RemoveCheckpoint(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) + err := cpdb.RemoveCheckpoint(ctx, "`db1`.`t2`") + require.NoError(t, err) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusLoaded) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusLoaded, cp.Status) } -func (s *cpFileSuite) TestIgnoreAllErrorCheckpoints(c *C) { +func TestIgnoreAllErrorCheckpoints(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - s.setInvalidStatus() + setInvalidStatus(cpdb) - err := s.cpdb.IgnoreErrorCheckpoint(ctx, "all") - c.Assert(err, IsNil) + err := cpdb.IgnoreErrorCheckpoint(ctx, "all") + require.NoError(t, err) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusLoaded) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusLoaded, cp.Status) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusLoaded) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusLoaded, cp.Status) } -func (s *cpFileSuite) TestIgnoreOneErrorCheckpoints(c *C) { +func TestIgnoreOneErrorCheckpoints(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - s.setInvalidStatus() + setInvalidStatus(cpdb) - err := s.cpdb.IgnoreErrorCheckpoint(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) + err := cpdb.IgnoreErrorCheckpoint(ctx, "`db1`.`t2`") + require.NoError(t, err) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusLoaded) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusLoaded, cp.Status) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusAllWritten/10) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusAllWritten/10, cp.Status) } -func (s *cpFileSuite) TestDestroyAllErrorCheckpoints(c *C) { +func TestDestroyAllErrorCheckpoints(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - s.setInvalidStatus() + setInvalidStatus(cpdb) - dtc, err := s.cpdb.DestroyErrorCheckpoint(ctx, "all") - c.Assert(err, IsNil) + dtc, err := cpdb.DestroyErrorCheckpoint(ctx, "all") + require.NoError(t, err) sort.Slice(dtc, func(i, j int) bool { return dtc[i].TableName < dtc[j].TableName }) - c.Assert(dtc, DeepEquals, []checkpoints.DestroyedTableCheckpoint{ + expect := []checkpoints.DestroyedTableCheckpoint{ { TableName: "`db1`.`t2`", MinEngineID: -1, @@ -293,37 +295,41 @@ func (s *cpFileSuite) TestDestroyAllErrorCheckpoints(c *C) { MinEngineID: -1, MaxEngineID: -1, }, - }) + } + require.Equal(t, expect, dtc) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) } -func (s *cpFileSuite) TestDestroyOneErrorCheckpoint(c *C) { +func TestDestroyOneErrorCheckpoint(t *testing.T) { ctx := context.Background() + cpdb, clean := newFileCheckpointsDB(t) + defer clean() - s.setInvalidStatus() + setInvalidStatus(cpdb) - dtc, err := s.cpdb.DestroyErrorCheckpoint(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(dtc, DeepEquals, []checkpoints.DestroyedTableCheckpoint{ + dtc, err := cpdb.DestroyErrorCheckpoint(ctx, "`db1`.`t2`") + require.NoError(t, err) + expect := []checkpoints.DestroyedTableCheckpoint{ { TableName: "`db1`.`t2`", MinEngineID: -1, MaxEngineID: 0, }, - }) + } + require.Equal(t, expect, dtc) - cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + cp, err := cpdb.Get(ctx, "`db1`.`t2`") + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) - cp, err = s.cpdb.Get(ctx, "`db2`.`t3`") - c.Assert(err, IsNil) - c.Assert(cp.Status, Equals, checkpoints.CheckpointStatusAllWritten/10) + cp, err = cpdb.Get(ctx, "`db2`.`t3`") + require.NoError(t, err) + require.Equal(t, checkpoints.CheckpointStatusAllWritten/10, cp.Status) } diff --git a/br/pkg/lightning/checkpoints/checkpoints_sql_test.go b/br/pkg/lightning/checkpoints/checkpoints_sql_test.go index a27f62c72a3eb..85a23f379fec9 100644 --- a/br/pkg/lightning/checkpoints/checkpoints_sql_test.go +++ b/br/pkg/lightning/checkpoints/checkpoints_sql_test.go @@ -4,28 +4,28 @@ import ( "context" "database/sql" "strings" + "testing" "time" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/verification" "github.com/pingcap/tidb/br/pkg/version/build" + "github.com/stretchr/testify/require" ) -var _ = Suite(&cpSQLSuite{}) - type cpSQLSuite struct { db *sql.DB mock sqlmock.Sqlmock cpdb *checkpoints.MySQLCheckpointsDB } -func (s *cpSQLSuite) SetUpTest(c *C) { +func newCPSQLSuite(t *testing.T) (*cpSQLSuite, func()) { + var s cpSQLSuite db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) s.db = db s.mock = mock @@ -47,19 +47,20 @@ func (s *cpSQLSuite) SetUpTest(c *C) { WillReturnResult(sqlmock.NewResult(5, 1)) cpdb, err := checkpoints.NewMySQLCheckpointsDB(context.Background(), s.db, "mock-schema") - c.Assert(err, IsNil) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.Nil(t, s.mock.ExpectationsWereMet()) s.cpdb = cpdb + return &s, func() { + s.mock.ExpectClose() + require.Nil(t, s.cpdb.Close()) + require.Nil(t, s.mock.ExpectationsWereMet()) + } } -func (s *cpSQLSuite) TearDownTest(c *C) { - s.mock.ExpectClose() - c.Assert(s.cpdb.Close(), IsNil) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) -} - -func (s *cpSQLSuite) TestNormalOperations(c *C) { +func TestNormalOperations(t *testing.T) { ctx := context.Background() + s, clean := newCPSQLSuite(t) + defer clean() cpdb := s.cpdb // 2. initialize with checkpoint data. @@ -101,8 +102,8 @@ func (s *cpSQLSuite) TestNormalOperations(c *C) { }, }) s.mock.MatchExpectationsInOrder(true) - c.Assert(err, IsNil) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.Nil(t, s.mock.ExpectationsWereMet()) // 3. set some checkpoints @@ -154,8 +155,8 @@ func (s *cpSQLSuite) TestNormalOperations(c *C) { }, }) s.mock.MatchExpectationsInOrder(true) - c.Assert(err, IsNil) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.Nil(t, s.mock.ExpectationsWereMet()) // 4. update some checkpoints @@ -222,7 +223,7 @@ func (s *cpSQLSuite) TestNormalOperations(c *C) { s.mock.MatchExpectationsInOrder(false) cpdb.Update(map[string]*checkpoints.TableCheckpointDiff{"`db1`.`t2`": cpd}) s.mock.MatchExpectationsInOrder(true) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) + require.Nil(t, s.mock.ExpectationsWereMet()) // 5. get back the checkpoints @@ -260,8 +261,8 @@ func (s *cpSQLSuite) TestNormalOperations(c *C) { s.mock.ExpectCommit() cp, err := cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(cp, DeepEquals, &checkpoints.TableCheckpoint{ + require.Nil(t, err) + require.Equal(t, &checkpoints.TableCheckpoint{ Status: checkpoints.CheckpointStatusAllWritten, AllocBase: 132861, TableID: int64(2), @@ -292,17 +293,20 @@ func (s *cpSQLSuite) TestNormalOperations(c *C) { }, }, Checksum: verification.MakeKVChecksum(4492, 686, 486070148910), - }) - c.Assert(s.mock.ExpectationsWereMet(), IsNil) + }, cp) + require.Nil(t, s.mock.ExpectationsWereMet()) } -func (s *cpSQLSuite) TestRemoveAllCheckpoints(c *C) { +func TestRemoveAllCheckpoints_SQL(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectExec("DROP SCHEMA `mock-schema`").WillReturnResult(sqlmock.NewResult(0, 1)) ctx := context.Background() err := s.cpdb.RemoveCheckpoint(ctx, "all") - c.Assert(err, IsNil) + require.NoError(t, err) s.mock.ExpectBegin() s.mock. @@ -325,11 +329,14 @@ func (s *cpSQLSuite) TestRemoveAllCheckpoints(c *C) { s.mock.ExpectRollback() cp, err := s.cpdb.Get(ctx, "`db1`.`t2`") - c.Assert(cp, IsNil) - c.Assert(errors.IsNotFound(err), IsTrue) + require.Nil(t, cp) + require.True(t, errors.IsNotFound(err)) } -func (s *cpSQLSuite) TestRemoveOneCheckpoint(c *C) { +func TestRemoveOneCheckpoint_SQL(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectBegin() s.mock. ExpectExec("DELETE FROM `mock-schema`\\.chunk_v\\d+ WHERE table_name = \\?"). @@ -346,10 +353,13 @@ func (s *cpSQLSuite) TestRemoveOneCheckpoint(c *C) { s.mock.ExpectCommit() err := s.cpdb.RemoveCheckpoint(context.Background(), "`db1`.`t2`") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *cpSQLSuite) TestIgnoreAllErrorCheckpoints(c *C) { +func TestIgnoreAllErrorCheckpoints_SQL(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectBegin() s.mock. ExpectExec("UPDATE `mock-schema`\\.engine_v\\d+ SET status = 30 WHERE 'all' = \\? AND status <= 25"). @@ -362,10 +372,13 @@ func (s *cpSQLSuite) TestIgnoreAllErrorCheckpoints(c *C) { s.mock.ExpectCommit() err := s.cpdb.IgnoreErrorCheckpoint(context.Background(), "all") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *cpSQLSuite) TestIgnoreOneErrorCheckpoint(c *C) { +func TestIgnoreOneErrorCheckpoint(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectBegin() s.mock. ExpectExec("UPDATE `mock-schema`\\.engine_v\\d+ SET status = 30 WHERE table_name = \\? AND status <= 25"). @@ -378,10 +391,13 @@ func (s *cpSQLSuite) TestIgnoreOneErrorCheckpoint(c *C) { s.mock.ExpectCommit() err := s.cpdb.IgnoreErrorCheckpoint(context.Background(), "`db1`.`t2`") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *cpSQLSuite) TestDestroyAllErrorCheckpoints(c *C) { +func TestDestroyAllErrorCheckpoints_SQL(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectBegin() s.mock. ExpectQuery("SELECT (?s:.+)'all' = \\?"). @@ -405,15 +421,18 @@ func (s *cpSQLSuite) TestDestroyAllErrorCheckpoints(c *C) { s.mock.ExpectCommit() dtc, err := s.cpdb.DestroyErrorCheckpoint(context.Background(), "all") - c.Assert(err, IsNil) - c.Assert(dtc, DeepEquals, []checkpoints.DestroyedTableCheckpoint{{ + require.NoError(t, err) + require.Equal(t, []checkpoints.DestroyedTableCheckpoint{{ TableName: "`db1`.`t2`", MinEngineID: -1, MaxEngineID: 0, - }}) + }}, dtc) } -func (s *cpSQLSuite) TestDestroyOneErrorCheckpoints(c *C) { +func TestDestroyOneErrorCheckpoints(t *testing.T) { + s, clean := newCPSQLSuite(t) + defer clean() + s.mock.ExpectBegin() s.mock. ExpectQuery("SELECT (?s:.+)table_name = \\?"). @@ -437,17 +456,19 @@ func (s *cpSQLSuite) TestDestroyOneErrorCheckpoints(c *C) { s.mock.ExpectCommit() dtc, err := s.cpdb.DestroyErrorCheckpoint(context.Background(), "`db1`.`t2`") - c.Assert(err, IsNil) - c.Assert(dtc, DeepEquals, []checkpoints.DestroyedTableCheckpoint{{ + require.NoError(t, err) + require.Equal(t, []checkpoints.DestroyedTableCheckpoint{{ TableName: "`db1`.`t2`", MinEngineID: -1, MaxEngineID: 0, - }}) + }}, dtc) } -func (s *cpSQLSuite) TestDump(c *C) { +func TestDump(t *testing.T) { ctx := context.Background() - t := time.Unix(1555555555, 0).UTC() + s, clean := newCPSQLSuite(t) + defer clean() + tm := time.Unix(1555555555, 0).UTC() s.mock. ExpectQuery("SELECT (?s:.+) FROM `mock-schema`\\.chunk_v\\d+"). @@ -461,53 +482,55 @@ func (s *cpSQLSuite) TestDump(c *C) { "`db1`.`t2`", "/tmp/path/1.sql", 0, mydump.SourceTypeSQL, mydump.CompressionNone, "", 456, "[]", 55904, 102400, 681, 5000, 4491, 586, 486070148917, - t, t, + tm, tm, ), ) var csvBuilder strings.Builder err := s.cpdb.DumpChunks(ctx, &csvBuilder) - c.Assert(err, IsNil) - c.Assert(csvBuilder.String(), Equals, + require.NoError(t, err) + require.Equal(t, "table_name,path,offset,type,compression,sort_key,file_size,columns,pos,end_offset,prev_rowid_max,rowid_max,kvc_bytes,kvc_kvs,kvc_checksum,create_time,update_time\n"+ "`db1`.`t2`,/tmp/path/1.sql,0,3,0,,456,[],55904,102400,681,5000,4491,586,486070148917,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n", + csvBuilder.String(), ) s.mock. ExpectQuery("SELECT .+ FROM `mock-schema`\\.engine_v\\d+"). WillReturnRows( sqlmock.NewRows([]string{"table_name", "engine_id", "status", "create_time", "update_time"}). - AddRow("`db1`.`t2`", -1, 30, t, t). - AddRow("`db1`.`t2`", 0, 120, t, t), + AddRow("`db1`.`t2`", -1, 30, tm, tm). + AddRow("`db1`.`t2`", 0, 120, tm, tm), ) csvBuilder.Reset() err = s.cpdb.DumpEngines(ctx, &csvBuilder) - c.Assert(err, IsNil) - c.Assert(csvBuilder.String(), Equals, - "table_name,engine_id,status,create_time,update_time\n"+ - "`db1`.`t2`,-1,30,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n"+ - "`db1`.`t2`,0,120,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n", - ) + require.NoError(t, err) + require.Equal(t, "table_name,engine_id,status,create_time,update_time\n"+ + "`db1`.`t2`,-1,30,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n"+ + "`db1`.`t2`,0,120,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n", + csvBuilder.String()) s.mock. ExpectQuery("SELECT .+ FROM `mock-schema`\\.table_v\\d+"). WillReturnRows( sqlmock.NewRows([]string{"task_id", "table_name", "hash", "status", "alloc_base", "create_time", "update_time"}). - AddRow(1555555555, "`db1`.`t2`", 0, 90, 132861, t, t), + AddRow(1555555555, "`db1`.`t2`", 0, 90, 132861, tm, tm), ) csvBuilder.Reset() err = s.cpdb.DumpTables(ctx, &csvBuilder) - c.Assert(err, IsNil) - c.Assert(csvBuilder.String(), Equals, - "task_id,table_name,hash,status,alloc_base,create_time,update_time\n"+ - "1555555555,`db1`.`t2`,0,90,132861,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n", + require.NoError(t, err) + require.Equal(t, "task_id,table_name,hash,status,alloc_base,create_time,update_time\n"+ + "1555555555,`db1`.`t2`,0,90,132861,2019-04-18 02:45:55 +0000 UTC,2019-04-18 02:45:55 +0000 UTC\n", + csvBuilder.String(), ) } -func (s *cpSQLSuite) TestMoveCheckpoints(c *C) { +func TestMoveCheckpoints(t *testing.T) { ctx := context.Background() + s, clean := newCPSQLSuite(t) + defer clean() s.mock. ExpectExec("CREATE SCHEMA IF NOT EXISTS `mock-schema\\.12345678\\.bak`"). @@ -526,5 +549,5 @@ func (s *cpSQLSuite) TestMoveCheckpoints(c *C) { WillReturnResult(sqlmock.NewResult(0, 1)) err := s.cpdb.MoveCheckpoints(ctx, 12345678) - c.Assert(err, IsNil) + require.NoError(t, err) } diff --git a/br/pkg/lightning/checkpoints/checkpoints_test.go b/br/pkg/lightning/checkpoints/checkpoints_test.go index e386fb59d2965..da15d1ad2c55a 100644 --- a/br/pkg/lightning/checkpoints/checkpoints_test.go +++ b/br/pkg/lightning/checkpoints/checkpoints_test.go @@ -4,28 +4,19 @@ import ( "path/filepath" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints/checkpointspb" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/verification" + "github.com/stretchr/testify/require" ) -func Test(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&checkpointSuite{}) - -type checkpointSuite struct { -} - -func (s *checkpointSuite) TestMergeStatusCheckpoint(c *C) { +func TestMergeStatusCheckpoint(t *testing.T) { cpd := NewTableCheckpointDiff() m := StatusCheckpointMerger{EngineID: 0, Status: CheckpointStatusImported} m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasStatus: false, engines: map[int32]engineCheckpointDiff{ 0: { @@ -34,12 +25,12 @@ func (s *checkpointSuite) TestMergeStatusCheckpoint(c *C) { chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), }, }, - }) + }, cpd) m = StatusCheckpointMerger{EngineID: -1, Status: CheckpointStatusLoaded} m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasStatus: false, engines: map[int32]engineCheckpointDiff{ 0: { @@ -53,12 +44,12 @@ func (s *checkpointSuite) TestMergeStatusCheckpoint(c *C) { chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), }, }, - }) + }, cpd) m = StatusCheckpointMerger{EngineID: WholeTableEngineID, Status: CheckpointStatusClosed} m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasStatus: true, status: CheckpointStatusClosed, engines: map[int32]engineCheckpointDiff{ @@ -73,12 +64,12 @@ func (s *checkpointSuite) TestMergeStatusCheckpoint(c *C) { chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), }, }, - }) + }, cpd) m = StatusCheckpointMerger{EngineID: -1, Status: CheckpointStatusAllWritten} m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasStatus: true, status: CheckpointStatusClosed, engines: map[int32]engineCheckpointDiff{ @@ -93,10 +84,10 @@ func (s *checkpointSuite) TestMergeStatusCheckpoint(c *C) { chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), }, }, - }) + }, cpd) } -func (s *checkpointSuite) TestMergeInvalidStatusCheckpoint(c *C) { +func TestMergeInvalidStatusCheckpoint(t *testing.T) { cpd := NewTableCheckpointDiff() m := StatusCheckpointMerger{EngineID: 0, Status: CheckpointStatusLoaded} @@ -106,7 +97,7 @@ func (s *checkpointSuite) TestMergeInvalidStatusCheckpoint(c *C) { m.SetInvalid() m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasStatus: true, status: CheckpointStatusAllWritten / 10, engines: map[int32]engineCheckpointDiff{ @@ -121,10 +112,10 @@ func (s *checkpointSuite) TestMergeInvalidStatusCheckpoint(c *C) { chunks: make(map[ChunkCheckpointKey]chunkCheckpointDiff), }, }, - }) + }, cpd) } -func (s *checkpointSuite) TestMergeChunkCheckpoint(c *C) { +func TestMergeChunkCheckpoint(t *testing.T) { cpd := NewTableCheckpointDiff() key := ChunkCheckpointKey{Path: "/tmp/path/1.sql", Offset: 0} @@ -138,7 +129,7 @@ func (s *checkpointSuite) TestMergeChunkCheckpoint(c *C) { } m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ engines: map[int32]engineCheckpointDiff{ 2: { chunks: map[ChunkCheckpointKey]chunkCheckpointDiff{ @@ -150,7 +141,7 @@ func (s *checkpointSuite) TestMergeChunkCheckpoint(c *C) { }, }, }, - }) + }, cpd) m = ChunkCheckpointMerger{ EngineID: 2, @@ -161,7 +152,7 @@ func (s *checkpointSuite) TestMergeChunkCheckpoint(c *C) { } m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ engines: map[int32]engineCheckpointDiff{ 2: { chunks: map[ChunkCheckpointKey]chunkCheckpointDiff{ @@ -173,23 +164,23 @@ func (s *checkpointSuite) TestMergeChunkCheckpoint(c *C) { }, }, }, - }) + }, cpd) } -func (s *checkpointSuite) TestRebaseCheckpoint(c *C) { +func TestRebaseCheckpoint(t *testing.T) { cpd := NewTableCheckpointDiff() m := RebaseCheckpointMerger{AllocBase: 10000} m.MergeInto(cpd) - c.Assert(cpd, DeepEquals, &TableCheckpointDiff{ + require.Equal(t, &TableCheckpointDiff{ hasRebase: true, allocBase: 10000, engines: make(map[int32]engineCheckpointDiff), - }) + }, cpd) } -func (s *checkpointSuite) TestApplyDiff(c *C) { +func TestApplyDiff(t *testing.T) { cp := TableCheckpoint{ Status: CheckpointStatusLoaded, AllocBase: 123, @@ -256,7 +247,7 @@ func (s *checkpointSuite) TestApplyDiff(c *C) { cp.Apply(cpd) - c.Assert(cp, DeepEquals, TableCheckpoint{ + require.Equal(t, TableCheckpoint{ Status: CheckpointStatusAllWritten, AllocBase: 11111, Engines: map[int32]*EngineCheckpoint{ @@ -288,11 +279,12 @@ func (s *checkpointSuite) TestApplyDiff(c *C) { }, }, }, - }) + }, cp) } -func (s *checkpointSuite) TestCheckpointMarshallUnmarshall(c *C) { - path := filepath.Join(c.MkDir(), "filecheckpoint") +func TestCheckpointMarshallUnmarshall(t *testing.T) { + dir := t.TempDir() + path := filepath.Join(dir, "filecheckpoint") fileChkp := NewFileCheckpointsDB(path) fileChkp.checkpoints.Checkpoints["a"] = &checkpointspb.TableCheckpointModel{ Status: uint32(CheckpointStatusLoaded), @@ -302,5 +294,5 @@ func (s *checkpointSuite) TestCheckpointMarshallUnmarshall(c *C) { fileChkp2 := NewFileCheckpointsDB(path) // if not recover empty map explicitly, it will become nil - c.Assert(fileChkp2.checkpoints.Checkpoints["a"].Engines, NotNil) + require.NotNil(t, fileChkp2.checkpoints.Checkpoints["a"].Engines) } diff --git a/br/pkg/lightning/checkpoints/main_test.go b/br/pkg/lightning/checkpoints/main_test.go new file mode 100644 index 0000000000000..43a529e2dc1bf --- /dev/null +++ b/br/pkg/lightning/checkpoints/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package checkpoints_test + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/br/pkg/lightning/common/conn.go b/br/pkg/lightning/common/conn.go index 83e6d3307412d..eb9b598e64c55 100644 --- a/br/pkg/lightning/common/conn.go +++ b/br/pkg/lightning/common/conn.go @@ -72,10 +72,10 @@ func (p *ConnPool) get(ctx context.Context) (*grpc.ClientConn, error) { } // NewConnPool creates a new connPool by the specified conn factory function and capacity. -func NewConnPool(cap int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *ConnPool { +func NewConnPool(capacity int, newConn func(ctx context.Context) (*grpc.ClientConn, error)) *ConnPool { return &ConnPool{ - cap: cap, - conns: make([]*grpc.ClientConn, 0, cap), + cap: capacity, + conns: make([]*grpc.ClientConn, 0, capacity), newConn: newConn, mu: sync.Mutex{}, diff --git a/br/pkg/kv/main_test.go b/br/pkg/lightning/common/main_test.go similarity index 78% rename from br/pkg/kv/main_test.go rename to br/pkg/lightning/common/main_test.go index 72bbbcfbf4cb0..8182878986d18 100644 --- a/br/pkg/kv/main_test.go +++ b/br/pkg/lightning/common/main_test.go @@ -1,4 +1,4 @@ -// Copyright 2021 PingCAP, Inc. +// Copyright 2022 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package kv +package common_test import ( "testing" @@ -23,5 +23,8 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() - goleak.VerifyTestMain(m) + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/lightning/common/once_error_test.go b/br/pkg/lightning/common/once_error_test.go index 3e4ae41121a11..1ba86d0a8b2e9 100644 --- a/br/pkg/lightning/common/once_error_test.go +++ b/br/pkg/lightning/common/once_error_test.go @@ -18,36 +18,28 @@ import ( "errors" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/stretchr/testify/require" ) -func TestCommon(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&onceErrorSuite{}) - -type onceErrorSuite struct{} - -func (s *onceErrorSuite) TestOnceError(c *C) { +func TestOnceError(t *testing.T) { var err common.OnceError - c.Assert(err.Get(), IsNil) + require.Nil(t, err.Get()) err.Set(nil) - c.Assert(err.Get(), IsNil) + require.Nil(t, err.Get()) e := errors.New("1") err.Set(e) - c.Assert(err.Get(), Equals, e) + require.Equal(t, e, err.Get()) e2 := errors.New("2") err.Set(e2) - c.Assert(err.Get(), Equals, e) // e, not e2. + require.Equal(t, e, err.Get()) // e, not e2. err.Set(nil) - c.Assert(err.Get(), Equals, e) + require.Equal(t, e, err.Get()) ch := make(chan struct{}) go func() { @@ -55,5 +47,5 @@ func (s *onceErrorSuite) TestOnceError(c *C) { ch <- struct{}{} }() <-ch - c.Assert(err.Get(), Equals, e) + require.Equal(t, e, err.Get()) } diff --git a/br/pkg/lightning/common/pause_test.go b/br/pkg/lightning/common/pause_test.go index 34ee124b7026a..8676482dc738b 100644 --- a/br/pkg/lightning/common/pause_test.go +++ b/br/pkg/lightning/common/pause_test.go @@ -17,27 +17,15 @@ package common_test import ( "context" "sync" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -// unblocksAfter is a checker which ensures the WaitGroup's Wait() method -// returns between the given durations. -var unblocksBetween Checker = &unblocksChecker{ - &CheckerInfo{Name: "unblocksBetween", Params: []string{"waitGroupPtr", "min", "max"}}, -} - -type unblocksChecker struct { - *CheckerInfo -} - -func (checker *unblocksChecker) Check(params []interface{}, names []string) (bool, string) { - wg := params[0].(*sync.WaitGroup) - min := params[1].(time.Duration) - max := params[2].(time.Duration) - +func assertUnblocksBetween(t *testing.T, wg *sync.WaitGroup, min, max time.Duration) { ch := make(chan time.Duration) start := time.Now() go func() { @@ -47,24 +35,19 @@ func (checker *unblocksChecker) Check(params []interface{}, names []string) (boo select { case dur := <-ch: if dur < min { - return false, "WaitGroup unblocked before minimum duration, it was " + dur.String() + t.Fatal("WaitGroup unblocked before minimum duration, it was " + dur.String()) } - return true, "" case <-time.After(max): select { case dur := <-ch: - return false, "WaitGroup did not unblock after maximum duration, it was " + dur.String() + t.Fatal("WaitGroup did not unblock after maximum duration, it was " + dur.String()) case <-time.After(1 * time.Second): - return false, "WaitGroup did not unblock after maximum duration" + t.Fatal("WaitGroup did not unblock after maximum duration") } } } -var _ = Suite(&pauseSuite{}) - -type pauseSuite struct{} - -func (s *pauseSuite) TestPause(c *C) { +func TestPause(t *testing.T) { var wg sync.WaitGroup p := common.NewPauser() @@ -75,12 +58,12 @@ func (s *pauseSuite) TestPause(c *C) { go func() { defer wg.Done() err := p.Wait(context.Background()) - c.Assert(err, IsNil) + assert.NoError(t, err) }() } // Give them more time to unblock in case of time exceeding due to high pressure of CI. - c.Assert(&wg, unblocksBetween, 0*time.Millisecond, 100*time.Millisecond) + assertUnblocksBetween(t, &wg, 0*time.Millisecond, 100*time.Millisecond) // after calling Pause(), these should be blocking... @@ -91,7 +74,7 @@ func (s *pauseSuite) TestPause(c *C) { go func() { defer wg.Done() err := p.Wait(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) }() } @@ -103,7 +86,7 @@ func (s *pauseSuite) TestPause(c *C) { }() // Give them more time to unblock in case of time exceeding due to high pressure of CI. - c.Assert(&wg, unblocksBetween, 500*time.Millisecond, 800*time.Millisecond) + assertUnblocksBetween(t, &wg, 500*time.Millisecond, 800*time.Millisecond) // if the context is canceled, Wait() should immediately unblock... @@ -116,13 +99,13 @@ func (s *pauseSuite) TestPause(c *C) { go func() { defer wg.Done() err := p.Wait(ctx) - c.Assert(err, Equals, context.Canceled) + require.ErrorIs(t, err, context.Canceled) }() } cancel() // Give them more time to unblock in case of time exceeding due to high pressure of CI. - c.Assert(&wg, unblocksBetween, 0*time.Millisecond, 100*time.Millisecond) + assertUnblocksBetween(t, &wg, 0*time.Millisecond, 100*time.Millisecond) // canceling the context does not affect the state of the pauser @@ -130,7 +113,7 @@ func (s *pauseSuite) TestPause(c *C) { go func() { defer wg.Done() err := p.Wait(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) }() go func() { @@ -139,29 +122,29 @@ func (s *pauseSuite) TestPause(c *C) { }() // Give them more time to unblock in case of time exceeding due to high pressure of CI. - c.Assert(&wg, unblocksBetween, 500*time.Millisecond, 800*time.Millisecond) + assertUnblocksBetween(t, &wg, 500*time.Millisecond, 800*time.Millisecond) } // Run `go test github.com/pingcap/tidb/br/pkg/lightning/common -check.b -test.v` to get benchmark result. -func (s *pauseSuite) BenchmarkWaitNoOp(c *C) { +func BenchmarkWaitNoOp(b *testing.B) { p := common.NewPauser() ctx := context.Background() - for i := 0; i < c.N; i++ { + for i := 0; i < b.N; i++ { _ = p.Wait(ctx) } } -func (s *pauseSuite) BenchmarkWaitCtxCanceled(c *C) { +func BenchmarkWaitCtxCanceled(b *testing.B) { p := common.NewPauser() p.Pause() ctx, cancel := context.WithCancel(context.Background()) cancel() - for i := 0; i < c.N; i++ { + for i := 0; i < b.N; i++ { _ = p.Wait(ctx) } } -func (s *pauseSuite) BenchmarkWaitContended(c *C) { +func BenchmarkWaitContended(b *testing.B) { p := common.NewPauser() done := make(chan struct{}) @@ -184,7 +167,7 @@ func (s *pauseSuite) BenchmarkWaitContended(c *C) { }() ctx := context.Background() - for i := 0; i < c.N; i++ { + for i := 0; i < b.N; i++ { _ = p.Wait(ctx) } } diff --git a/br/pkg/lightning/common/security.go b/br/pkg/lightning/common/security.go index 08718ce1d1406..9db53f78a5115 100644 --- a/br/pkg/lightning/common/security.go +++ b/br/pkg/lightning/common/security.go @@ -72,10 +72,11 @@ func ToTLSConfig(caPath, certPath, keyPath string) (*tls.Config, error) { return nil, errors.New("failed to append ca certs") } - return &tls.Config{ // nolint:gosec + return &tls.Config{ Certificates: certificates, RootCAs: certPool, NextProtos: []string{"h2", "http/1.1"}, // specify `h2` to let Go use HTTP/2. + MinVersion: tls.VersionTLS12, }, nil } diff --git a/br/pkg/lightning/common/security_test.go b/br/pkg/lightning/common/security_test.go index 1eec0ef9f72ce..a68b8e1f4437a 100644 --- a/br/pkg/lightning/common/security_test.go +++ b/br/pkg/lightning/common/security_test.go @@ -22,42 +22,39 @@ import ( "net/url" "os" "path/filepath" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/stretchr/testify/require" ) -type securitySuite struct{} - -var _ = Suite(&securitySuite{}) - func respondPathHandler(w http.ResponseWriter, req *http.Request) { _, _ = io.WriteString(w, `{"path":"`) _, _ = io.WriteString(w, req.URL.Path) _, _ = io.WriteString(w, `"}`) } -func (s *securitySuite) TestGetJSONInsecure(c *C) { +func TestGetJSONInsecure(t *testing.T) { mockServer := httptest.NewServer(http.HandlerFunc(respondPathHandler)) defer mockServer.Close() ctx := context.Background() u, err := url.Parse(mockServer.URL) - c.Assert(err, IsNil) + require.NoError(t, err) tls, err := common.NewTLS("", "", "", u.Host) - c.Assert(err, IsNil) + require.NoError(t, err) var result struct{ Path string } err = tls.GetJSON(ctx, "/aaa", &result) - c.Assert(err, IsNil) - c.Assert(result.Path, Equals, "/aaa") + require.NoError(t, err) + require.Equal(t, "/aaa", result.Path) err = tls.GetJSON(ctx, "/bbbb", &result) - c.Assert(err, IsNil) - c.Assert(result.Path, Equals, "/bbbb") + require.NoError(t, err) + require.Equal(t, "/bbbb", result.Path) } -func (s *securitySuite) TestGetJSONSecure(c *C) { +func TestGetJSONSecure(t *testing.T) { mockServer := httptest.NewTLSServer(http.HandlerFunc(respondPathHandler)) defer mockServer.Close() @@ -66,34 +63,33 @@ func (s *securitySuite) TestGetJSONSecure(c *C) { var result struct{ Path string } err := tls.GetJSON(ctx, "/ccc", &result) - c.Assert(err, IsNil) - c.Assert(result.Path, Equals, "/ccc") + require.NoError(t, err) + require.Equal(t, "/ccc", result.Path) err = tls.GetJSON(ctx, "/dddd", &result) - c.Assert(err, IsNil) - c.Assert(result.Path, Equals, "/dddd") + require.NoError(t, err) + require.Equal(t, "/dddd", result.Path) } -func (s *securitySuite) TestInvalidTLS(c *C) { - tempDir := c.MkDir() - +func TestInvalidTLS(t *testing.T) { + tempDir := t.TempDir() caPath := filepath.Join(tempDir, "ca.pem") _, err := common.NewTLS(caPath, "", "", "localhost") - c.Assert(err, ErrorMatches, "could not read ca certificate:.*") + require.Regexp(t, "could not read ca certificate:.*", err.Error()) err = os.WriteFile(caPath, []byte("invalid ca content"), 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = common.NewTLS(caPath, "", "", "localhost") - c.Assert(err, ErrorMatches, "failed to append ca certs") + require.Regexp(t, "failed to append ca certs", err.Error()) certPath := filepath.Join(tempDir, "test.pem") keyPath := filepath.Join(tempDir, "test.key") _, err = common.NewTLS(caPath, certPath, keyPath, "localhost") - c.Assert(err, ErrorMatches, "could not load client key pair: open.*") + require.Regexp(t, "could not load client key pair: open.*", err.Error()) err = os.WriteFile(certPath, []byte("invalid cert content"), 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) err = os.WriteFile(keyPath, []byte("invalid key content"), 0o600) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = common.NewTLS(caPath, certPath, keyPath, "localhost") - c.Assert(err, ErrorMatches, "could not load client key pair: tls.*") + require.Regexp(t, "could not load client key pair: tls.*", err.Error()) } diff --git a/br/pkg/lightning/common/storage_test.go b/br/pkg/lightning/common/storage_test.go index db46269149a4c..11dc364c52e13 100644 --- a/br/pkg/lightning/common/storage_test.go +++ b/br/pkg/lightning/common/storage_test.go @@ -15,20 +15,17 @@ package common_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testStorageSuite{}) - -type testStorageSuite struct { -} - -func (t *testStorageSuite) TestGetStorageSize(c *C) { +func TestGetStorageSize(t *testing.T) { // only ensure we can get storage size. - d := c.MkDir() + d := t.TempDir() size, err := common.GetStorageSize(d) - c.Assert(err, IsNil) - c.Assert(size.Capacity, Greater, uint64(0)) - c.Assert(size.Available, Greater, uint64(0)) + require.NoError(t, err) + require.Greater(t, size.Capacity, uint64(0)) + require.Greater(t, size.Available, uint64(0)) } diff --git a/br/pkg/lightning/common/util_test.go b/br/pkg/lightning/common/util_test.go index 3668626276938..d06daa00b13f9 100644 --- a/br/pkg/lightning/common/util_test.go +++ b/br/pkg/lightning/common/util_test.go @@ -20,25 +20,23 @@ import ( "io" "net/http" "net/http/httptest" + "testing" "time" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -type utilSuite struct{} - -var _ = Suite(&utilSuite{}) - -func (s *utilSuite) TestDirNotExist(c *C) { - c.Assert(common.IsDirExists("."), IsTrue) - c.Assert(common.IsDirExists("not-exists"), IsFalse) +func TestDirNotExist(t *testing.T) { + require.True(t, common.IsDirExists(".")) + require.False(t, common.IsDirExists("not-exists")) } -func (s *utilSuite) TestGetJSON(c *C) { +func TestGetJSON(t *testing.T) { type TestPayload struct { Username string `json:"username"` Password string `json:"password"` @@ -53,7 +51,7 @@ func (s *utilSuite) TestGetJSON(c *C) { handle := func(res http.ResponseWriter, _ *http.Request) { res.WriteHeader(http.StatusOK) err := json.NewEncoder(res).Encode(request) - c.Assert(err, IsNil) + require.NoError(t, err) } testServer := httptest.NewServer(http.HandlerFunc(func(res http.ResponseWriter, req *http.Request) { handle(res, req) @@ -64,21 +62,21 @@ func (s *utilSuite) TestGetJSON(c *C) { response := TestPayload{} err := common.GetJSON(ctx, client, "http://not-exists", &response) - c.Assert(err, NotNil) + require.Error(t, err) err = common.GetJSON(ctx, client, testServer.URL, &response) - c.Assert(err, IsNil) - c.Assert(request, DeepEquals, response) + require.NoError(t, err) + require.Equal(t, request, response) // Mock `StatusNoContent` response handle = func(res http.ResponseWriter, _ *http.Request) { res.WriteHeader(http.StatusNoContent) } err = common.GetJSON(ctx, client, testServer.URL, &response) - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, ".*http status code != 200.*") + require.Error(t, err) + require.Regexp(t, ".*http status code != 200.*", err.Error()) } -func (s *utilSuite) TestToDSN(c *C) { +func TestToDSN(t *testing.T) { param := common.MySQLConnectParam{ Host: "127.0.0.1", Port: 4000, @@ -91,25 +89,26 @@ func (s *utilSuite) TestToDSN(c *C) { "tidb_distsql_scan_concurrency": "1", }, } - c.Assert(param.ToDSN(), Equals, "root:123456@tcp(127.0.0.1:4000)/?charset=utf8mb4&sql_mode='strict'&maxAllowedPacket=1234&tls=cluster&tidb_distsql_scan_concurrency='1'") + require.Equal(t, "root:123456@tcp(127.0.0.1:4000)/?charset=utf8mb4&sql_mode='strict'&maxAllowedPacket=1234&tls=cluster&tidb_distsql_scan_concurrency='1'", param.ToDSN()) } -func (s *utilSuite) TestIsContextCanceledError(c *C) { - c.Assert(common.IsContextCanceledError(context.Canceled), IsTrue) - c.Assert(common.IsContextCanceledError(io.EOF), IsFalse) +func TestIsContextCanceledError(t *testing.T) { + require.True(t, common.IsContextCanceledError(context.Canceled)) + require.False(t, common.IsContextCanceledError(io.EOF)) } -func (s *utilSuite) TestUniqueTable(c *C) { +func TestUniqueTable(t *testing.T) { tableName := common.UniqueTable("test", "t1") - c.Assert(tableName, Equals, "`test`.`t1`") + require.Equal(t, "`test`.`t1`", tableName) tableName = common.UniqueTable("test", "t`1") - c.Assert(tableName, Equals, "`test`.`t``1`") + require.Equal(t, "`test`.`t``1`", tableName) } -func (s *utilSuite) TestSQLWithRetry(c *C) { +func TestSQLWithRetry(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + defer db.Close() sqlWithRetry := &common.SQLWithRetry{ DB: db, @@ -122,48 +121,48 @@ func (s *utilSuite) TestSQLWithRetry(c *C) { mock.ExpectQuery("select a from test.t1").WillReturnError(errors.New("mock error")) } err = sqlWithRetry.QueryRow(context.Background(), "", "select a from test.t1", aValue) - c.Assert(err, ErrorMatches, ".*mock error") + require.Regexp(t, ".*mock error", err.Error()) // meet unretryable error and will return directly mock.ExpectQuery("select a from test.t1").WillReturnError(context.Canceled) err = sqlWithRetry.QueryRow(context.Background(), "", "select a from test.t1", aValue) - c.Assert(err, ErrorMatches, ".*context canceled") + require.Regexp(t, ".*context canceled", err.Error()) // query success rows := sqlmock.NewRows([]string{"a"}).AddRow("1") mock.ExpectQuery("select a from test.t1").WillReturnRows(rows) err = sqlWithRetry.QueryRow(context.Background(), "", "select a from test.t1", aValue) - c.Assert(err, IsNil) - c.Assert(*aValue, Equals, 1) + require.NoError(t, err) + require.Equal(t, 1, *aValue) // test Exec mock.ExpectExec("delete from").WillReturnError(context.Canceled) err = sqlWithRetry.Exec(context.Background(), "", "delete from test.t1 where id = ?", 2) - c.Assert(err, ErrorMatches, ".*context canceled") + require.Regexp(t, ".*context canceled", err.Error()) mock.ExpectExec("delete from").WillReturnResult(sqlmock.NewResult(0, 1)) err = sqlWithRetry.Exec(context.Background(), "", "delete from test.t1 where id = ?", 2) - c.Assert(err, IsNil) + require.NoError(t, err) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.Nil(t, mock.ExpectationsWereMet()) } -func (s *utilSuite) TestStringSliceEqual(c *C) { - c.Assert(common.StringSliceEqual(nil, nil), IsTrue) - c.Assert(common.StringSliceEqual(nil, []string{}), IsTrue) - c.Assert(common.StringSliceEqual(nil, []string{"a"}), IsFalse) - c.Assert(common.StringSliceEqual([]string{"a"}, nil), IsFalse) - c.Assert(common.StringSliceEqual([]string{"a"}, []string{"a"}), IsTrue) - c.Assert(common.StringSliceEqual([]string{"a"}, []string{"b"}), IsFalse) - c.Assert(common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b", "c"}), IsTrue) - c.Assert(common.StringSliceEqual([]string{"a"}, []string{"a", "b", "c"}), IsFalse) - c.Assert(common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b"}), IsFalse) - c.Assert(common.StringSliceEqual([]string{"a", "x", "y"}, []string{"a", "y", "x"}), IsFalse) +func TestStringSliceEqual(t *testing.T) { + assert.True(t, common.StringSliceEqual(nil, nil)) + assert.True(t, common.StringSliceEqual(nil, []string{})) + assert.False(t, common.StringSliceEqual(nil, []string{"a"})) + assert.False(t, common.StringSliceEqual([]string{"a"}, nil)) + assert.True(t, common.StringSliceEqual([]string{"a"}, []string{"a"})) + assert.False(t, common.StringSliceEqual([]string{"a"}, []string{"b"})) + assert.True(t, common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b", "c"})) + assert.False(t, common.StringSliceEqual([]string{"a"}, []string{"a", "b", "c"})) + assert.False(t, common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b"})) + assert.False(t, common.StringSliceEqual([]string{"a", "x", "y"}, []string{"a", "y", "x"})) } -func (s *utilSuite) TestInterpolateMySQLString(c *C) { - c.Assert(common.InterpolateMySQLString("123"), Equals, "'123'") - c.Assert(common.InterpolateMySQLString("1'23"), Equals, "'1''23'") - c.Assert(common.InterpolateMySQLString("1'2''3"), Equals, "'1''2''''3'") +func TestInterpolateMySQLString(t *testing.T) { + assert.Equal(t, "'123'", common.InterpolateMySQLString("123")) + assert.Equal(t, "'1''23'", common.InterpolateMySQLString("1'23")) + assert.Equal(t, "'1''2''''3'", common.InterpolateMySQLString("1'2''3")) } diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index 02a8ec80b1f63..23318d4c32012 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -789,11 +789,11 @@ func CleanupMetas(ctx context.Context, cfg *config.Config, tableName string) err func UnsafeCloseEngine(ctx context.Context, importer backend.Backend, engine string) (*backend.ClosedEngine, error) { if index := strings.LastIndexByte(engine, ':'); index >= 0 { tableName := engine[:index] - engineID, err := strconv.Atoi(engine[index+1:]) // nolint:gosec + engineID, err := strconv.Atoi(engine[index+1:]) if err != nil { return nil, errors.Trace(err) } - ce, err := importer.UnsafeCloseEngine(ctx, nil, tableName, int32(engineID)) + ce, err := importer.UnsafeCloseEngine(ctx, nil, tableName, int32(engineID)) // #nosec G109 return ce, errors.Trace(err) } diff --git a/br/pkg/lightning/lightning_server_serial_test.go b/br/pkg/lightning/lightning_server_serial_test.go index 099299f7f7d4e..8863bf648cc1e 100644 --- a/br/pkg/lightning/lightning_server_serial_test.go +++ b/br/pkg/lightning/lightning_server_serial_test.go @@ -24,14 +24,19 @@ import ( "fmt" "net/http" "strings" + "sync" "testing" "time" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/web" "github.com/stretchr/testify/require" ) +// initProgressOnce is used to ensure init progress once to avoid data race. +var initProgressOnce sync.Once + type lightningServerSuite struct { lightning *Lightning taskCfgCh chan *config.Config @@ -39,6 +44,8 @@ type lightningServerSuite struct { } func createSuite(t *testing.T) (s *lightningServerSuite, clean func()) { + initProgressOnce.Do(web.EnableCurrentProgress) + cfg := config.NewGlobalConfig() cfg.TiDB.Host = "test.invalid" cfg.TiDB.Port = 4000 diff --git a/br/pkg/lightning/log/filter_test.go b/br/pkg/lightning/log/filter_test.go index 658ce4d94a656..1d0344f8db9d2 100644 --- a/br/pkg/lightning/log/filter_test.go +++ b/br/pkg/lightning/log/filter_test.go @@ -37,10 +37,6 @@ func TestFilter(t *testing.T) { logger.Warn("the message", zap.Int("number", 123456), zap.Ints("array", []int{7, 8, 9})) require.Len(t, buffer.Stripped(), 0) - // Fields won't trigger filter. - logger, buffer = log.MakeTestLogger(zap.WrapCore(func(c zapcore.Core) zapcore.Core { - return log.NewFilterCore(c, "github.com/pingcap/check/").With([]zap.Field{zap.String("a", "b")}) - }), zap.AddCaller()) logger.Warn("the message", zap.String("stack", "github.com/pingcap/tidb/br/")) require.Len(t, buffer.Stripped(), 0) } diff --git a/br/pkg/lightning/mydump/charset_convertor_test.go b/br/pkg/lightning/mydump/charset_convertor_test.go index cf091c09b142e..5165ec2b63175 100644 --- a/br/pkg/lightning/mydump/charset_convertor_test.go +++ b/br/pkg/lightning/mydump/charset_convertor_test.go @@ -17,17 +17,11 @@ package mydump import ( "io" "os" + "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testCharsetConvertorSuite{}) - -type testCharsetConvertorSuite struct{} - -func (s *testCharsetConvertorSuite) SetUpSuite(c *C) {} -func (s *testCharsetConvertorSuite) TearDownSuite(c *C) {} - const ( testUTF8DataFile = "./csv/utf8_test_file.csv" testGBKDataFile = "./csv/gb18030_test_file.csv" @@ -40,28 +34,28 @@ var ( invalidChar = []byte{0xff} // Invalid gb18030 char ) -func (s testCharsetConvertorSuite) TestCharsetConvertor(c *C) { +func TestCharsetConvertor(t *testing.T) { utf8Reader, err := os.Open(testUTF8DataFile) - c.Assert(err, IsNil) + require.NoError(t, err) utf8Data, err := io.ReadAll(utf8Reader) - c.Assert(err, IsNil) + require.NoError(t, err) gbkReader, err := os.Open(testGBKDataFile) - c.Assert(err, IsNil) + require.NoError(t, err) gbkData, err := io.ReadAll(gbkReader) - c.Assert(err, IsNil) + require.NoError(t, err) cc, err := NewCharsetConvertor("gb18030", "\ufffd") - c.Assert(err, IsNil) + require.NoError(t, err) gbkToUTF8Data, err := cc.Decode(string(gbkData)) - c.Assert(err, IsNil) - c.Assert(gbkToUTF8Data, DeepEquals, string(utf8Data)) + require.NoError(t, err) + require.Equal(t, string(utf8Data), gbkToUTF8Data) utf8ToGBKData, err := cc.Encode(string(normalCharUTF8MB4)) - c.Assert(err, IsNil) - c.Assert(utf8ToGBKData, DeepEquals, string(normalCharGB18030)) + require.NoError(t, err) + require.Equal(t, string(normalCharGB18030), utf8ToGBKData) } -func (s testCharsetConvertorSuite) TestInvalidCharReplace(c *C) { +func TestInvalidCharReplace(t *testing.T) { dataInvalidCharReplace := "😅😅😅" // Input: 你好invalid char你好 inputData := append(normalCharGB18030, invalidChar...) @@ -71,16 +65,16 @@ func (s testCharsetConvertorSuite) TestInvalidCharReplace(c *C) { expectedData = append(expectedData, normalCharUTF8MB4...) // Prepare the file data. - c.Assert(os.WriteFile(testTempDataFile, inputData, 0666), IsNil) - defer func() { c.Assert(os.Remove(testTempDataFile), IsNil) }() + require.NoError(t, os.WriteFile(testTempDataFile, inputData, 0666)) + defer func() { require.NoError(t, os.Remove(testTempDataFile)) }() gbkReader, err := os.Open(testTempDataFile) - c.Assert(err, IsNil) + require.NoError(t, err) gbkData, err := io.ReadAll(gbkReader) - c.Assert(err, IsNil) + require.NoError(t, err) cc, err := NewCharsetConvertor("gb18030", dataInvalidCharReplace) - c.Assert(err, IsNil) + require.NoError(t, err) gbkToUTF8Data, err := cc.Decode(string(gbkData)) - c.Assert(err, IsNil) - c.Assert(gbkToUTF8Data, DeepEquals, string(expectedData)) + require.NoError(t, err) + require.Equal(t, string(expectedData), gbkToUTF8Data) } diff --git a/br/pkg/lightning/mydump/csv_parser_test.go b/br/pkg/lightning/mydump/csv_parser_test.go index de183ed5fd5d0..8b712865cfed2 100644 --- a/br/pkg/lightning/mydump/csv_parser_test.go +++ b/br/pkg/lightning/mydump/csv_parser_test.go @@ -3,47 +3,31 @@ package mydump_test import ( "context" "encoding/csv" + "fmt" "io" "os" "path/filepath" "strings" + "testing" "unicode/utf8" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) -var _ = Suite(&testMydumpCSVParserSuite{}) +var ioWorkers = worker.NewPool(context.Background(), 5, "test_csv") -type testMydumpCSVParserSuite struct { - ioWorkers *worker.Pool -} - -func (s *testMydumpCSVParserSuite) SetUpSuite(c *C) { - s.ioWorkers = worker.NewPool(context.Background(), 5, "test_csv") -} -func (s *testMydumpCSVParserSuite) TearDownSuite(c *C) {} - -type assertPosEq struct { - *CheckerInfo -} - -var posEq = &assertPosEq{ - &CheckerInfo{Name: "posEq", Params: []string{"parser", "pos", "rowID"}}, -} - -func (checker *assertPosEq) Check(params []interface{}, names []string) (result bool, error string) { - parser := params[0].(mydump.Parser) +func assertPosEqual(t *testing.T, parser mydump.Parser, expectPos, expectRowID int64) { pos, rowID := parser.Pos() - expectedPos := int64(params[1].(int)) - expectedRowID := int64(params[2].(int)) - return pos == expectedPos && rowID == expectedRowID, "" + require.Equal(t, expectPos, pos) + require.Equal(t, expectRowID, rowID) } var nullDatum types.Datum @@ -53,32 +37,31 @@ type testCase struct { expected [][]types.Datum } -func (s *testMydumpCSVParserSuite) runTestCases(c *C, cfg *config.MydumperRuntime, blockBufSize int64, cases []testCase) { +func runTestCasesCSV(t *testing.T, cfg *config.MydumperRuntime, blockBufSize int64, cases []testCase) { for _, tc := range cases { charsetConvertor, err := mydump.NewCharsetConvertor(cfg.DataCharacterSet, cfg.DataInvalidCharReplace) - c.Assert(err, IsNil) - parser, err := mydump.NewCSVParser(&cfg.CSV, mydump.NewStringReader(tc.input), blockBufSize, s.ioWorkers, false, charsetConvertor) - c.Assert(err, IsNil) + assert.NoError(t, err) + parser, err := mydump.NewCSVParser(&cfg.CSV, mydump.NewStringReader(tc.input), blockBufSize, ioWorkers, false, charsetConvertor) + assert.NoError(t, err) for i, row := range tc.expected { - comment := Commentf("input = %q, row = %d", tc.input, i+1) + comment := fmt.Sprintf("input = %q, row = %d", tc.input, i+1) e := parser.ReadRow() - c.Assert(e, IsNil, Commentf("input = %q, row = %d, error = %s", tc.input, i+1, errors.ErrorStack(e))) - c.Assert(parser.LastRow().RowID, DeepEquals, int64(i)+1, comment) - c.Assert(parser.LastRow().Row, DeepEquals, row, comment) - + assert.NoErrorf(t, e, "input = %q, row = %d, error = %s", tc.input, i+1, errors.ErrorStack(e)) + assert.Equal(t, int64(i)+1, parser.LastRow().RowID, comment) + assert.Equal(t, row, parser.LastRow().Row, comment) } - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF, Commentf("input = %q", tc.input)) + assert.ErrorIsf(t, errors.Cause(parser.ReadRow()), io.EOF, "input = %q", tc.input) } } -func (s *testMydumpCSVParserSuite) runFailingTestCases(c *C, cfg *config.MydumperRuntime, blockBufSize int64, cases []string) { +func runFailingTestCasesCSV(t *testing.T, cfg *config.MydumperRuntime, blockBufSize int64, cases []string) { for _, tc := range cases { charsetConvertor, err := mydump.NewCharsetConvertor(cfg.DataCharacterSet, cfg.DataInvalidCharReplace) - c.Assert(err, IsNil) - parser, err := mydump.NewCSVParser(&cfg.CSV, mydump.NewStringReader(tc), blockBufSize, s.ioWorkers, false, charsetConvertor) - c.Assert(err, IsNil) + assert.NoError(t, err) + parser, err := mydump.NewCSVParser(&cfg.CSV, mydump.NewStringReader(tc), blockBufSize, ioWorkers, false, charsetConvertor) + require.NoError(t, err) e := parser.ReadRow() - c.Assert(e, ErrorMatches, "syntax error.*", Commentf("input = %q / %s", tc, errors.ErrorStack(e))) + assert.Regexpf(t, "syntax error.*", e.Error(), "input = %q / %s", tc, errors.ErrorStack(e)) } } @@ -143,7 +126,7 @@ func datumsToString(datums [][]types.Datum, delimitor string, quote string, last return b.String() } -func (s *testMydumpCSVParserSuite) TestTPCH(c *C) { +func TestTPCH(t *testing.T) { datums := tpchDatums() input := datumsToString(datums, "|", "", true) reader := mydump.NewStringReader(input) @@ -154,37 +137,36 @@ func (s *testMydumpCSVParserSuite) TestTPCH(c *C) { TrimLastSep: true, } - parser, err := mydump.NewCSVParser(&cfg, reader, int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) - - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + parser, err := mydump.NewCSVParser(&cfg, reader, int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: datums[0], Length: 116, - }) - c.Assert(parser, posEq, 126, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 126, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: datums[1], Length: 104, - }) - c.Assert(parser, posEq, 241, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 241, 2) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 3, Row: datums[2], Length: 117, - }) - c.Assert(parser, posEq, 369, 3) + }, parser.LastRow()) + assertPosEqual(t, parser, 369, 3) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestTPCHMultiBytes(c *C) { +func TestTPCHMultiBytes(t *testing.T) { datums := tpchDatums() sepsAndQuotes := [][2]string{ {",", ""}, @@ -224,7 +206,7 @@ func (s *testMydumpCSVParserSuite) TestTPCHMultiBytes(c *C) { } allExpectedParserPos = append(allExpectedParserPos, last+pos+1) } - c.Assert(allExpectedParserPos, HasLen, len(datums)) + require.Len(t, allExpectedParserPos, len(datums)) cfg := config.CSVConfig{ Separator: SepAndQuote[0], @@ -233,22 +215,21 @@ func (s *testMydumpCSVParserSuite) TestTPCHMultiBytes(c *C) { } reader := mydump.NewStringReader(inputStr) - parser, err := mydump.NewCSVParser(&cfg, reader, int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err := mydump.NewCSVParser(&cfg, reader, int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) for i, expectedParserPos := range allExpectedParserPos { - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow().RowID, DeepEquals, int64(i+1)) - c.Assert(parser.LastRow().Row, DeepEquals, datums[i]) - - c.Assert(parser, posEq, expectedParserPos, i+1) + require.Nil(t, parser.ReadRow()) + require.Equal(t, int64(i+1), parser.LastRow().RowID) + require.Equal(t, datums[i], parser.LastRow().Row) + assertPosEqual(t, parser, int64(expectedParserPos), int64(i+1)) } - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } } -func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { +func TestRFC4180(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, @@ -256,11 +237,11 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { // example 1, trailing new lines - parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader("aaa,bbb,ccc\nzzz,yyy,xxx\n"), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader("aaa,bbb,ccc\nzzz,yyy,xxx\n"), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("aaa"), @@ -268,11 +249,11 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("ccc"), }, Length: 9, - }) - c.Assert(parser, posEq, 12, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 12, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("zzz"), @@ -280,18 +261,18 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("xxx"), }, Length: 9, - }) - c.Assert(parser, posEq, 24, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 24, 2) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) // example 2, no trailing new lines - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader("aaa,bbb,ccc\nzzz,yyy,xxx"), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader("aaa,bbb,ccc\nzzz,yyy,xxx"), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("aaa"), @@ -299,11 +280,11 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("ccc"), }, Length: 9, - }) - c.Assert(parser, posEq, 12, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 12, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("zzz"), @@ -311,18 +292,18 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("xxx"), }, Length: 9, - }) - c.Assert(parser, posEq, 23, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 23, 2) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) // example 5, quoted fields - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"aaa","bbb","ccc"`+"\nzzz,yyy,xxx"), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"aaa","bbb","ccc"`+"\nzzz,yyy,xxx"), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("aaa"), @@ -330,11 +311,11 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("ccc"), }, Length: 9, - }) - c.Assert(parser, posEq, 18, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 18, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("zzz"), @@ -342,20 +323,20 @@ func (s *testMydumpCSVParserSuite) TestRFC4180(c *C) { types.NewStringDatum("xxx"), }, Length: 9, - }) - c.Assert(parser, posEq, 29, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 29, 2) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) // example 6, line breaks within fields parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"aaa","b bb","ccc" -zzz,yyy,xxx`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) +zzz,yyy,xxx`), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("aaa"), @@ -363,11 +344,11 @@ zzz,yyy,xxx`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) types.NewStringDatum("ccc"), }, Length: 10, - }) - c.Assert(parser, posEq, 19, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 19, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("zzz"), @@ -375,18 +356,18 @@ zzz,yyy,xxx`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) types.NewStringDatum("xxx"), }, Length: 9, - }) - c.Assert(parser, posEq, 30, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 30, 2) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) // example 7, quote escaping - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"aaa","b""bb","ccc"`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"aaa","b""bb","ccc"`), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("aaa"), @@ -394,13 +375,13 @@ zzz,yyy,xxx`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) types.NewStringDatum("ccc"), }, Length: 10, - }) - c.Assert(parser, posEq, 19, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 19, 1) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestMySQL(c *C) { +func TestMySQL(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, @@ -411,11 +392,11 @@ func (s *testMydumpCSVParserSuite) TestMySQL(c *C) { parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(`"\"","\\","\?" "\ -",\N,\\N`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) +",\N,\\N`), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum(`"`), @@ -423,11 +404,11 @@ func (s *testMydumpCSVParserSuite) TestMySQL(c *C) { types.NewStringDatum("?"), }, Length: 6, - }) - c.Assert(parser, posEq, 15, 1) + }, parser.LastRow()) + assertPosEqual(t, parser, 15, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("\n"), @@ -435,13 +416,13 @@ func (s *testMydumpCSVParserSuite) TestMySQL(c *C) { types.NewStringDatum(`\N`), }, Length: 7, - }) - c.Assert(parser, posEq, 26, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 26, 2) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestSyntaxError(c *C) { +func TestSyntaxErrorCSV(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: ",", @@ -462,13 +443,13 @@ func (s *testMydumpCSVParserSuite) TestSyntaxError(c *C) { "\"\x01", } - s.runFailingTestCases(c, &cfg, int64(config.ReadBlockSize), inputs) + runFailingTestCasesCSV(t, &cfg, int64(config.ReadBlockSize), inputs) cfg.CSV.BackslashEscape = false - s.runFailingTestCases(c, &cfg, int64(config.ReadBlockSize), []string{`"\`}) + runFailingTestCasesCSV(t, &cfg, int64(config.ReadBlockSize), []string{`"\`}) } -func (s *testMydumpCSVParserSuite) TestTSV(c *C) { +func TestTSV(t *testing.T) { cfg := config.CSVConfig{ Separator: "\t", Delimiter: "", @@ -481,11 +462,11 @@ func (s *testMydumpCSVParserSuite) TestTSV(c *C) { parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(`a b c d e f 0 foo 0000-00-00 0 foo 0000-00-00 -0 abc def ghi bar 1999-12-31`), int64(config.ReadBlockSize), s.ioWorkers, true, nil) - c.Assert(err, IsNil) +0 abc def ghi bar 1999-12-31`), int64(config.ReadBlockSize), ioWorkers, true, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("0"), @@ -496,12 +477,12 @@ func (s *testMydumpCSVParserSuite) TestTSV(c *C) { types.NewStringDatum("0000-00-00"), }, Length: 14, - }) - c.Assert(parser, posEq, 32, 1) - c.Assert(parser.Columns(), DeepEquals, []string{"a", "b", "c", "d", "e", "f"}) + }, parser.LastRow()) + assertPosEqual(t, parser, 32, 1) + require.Equal(t, []string{"a", "b", "c", "d", "e", "f"}, parser.Columns()) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("0"), @@ -512,11 +493,11 @@ func (s *testMydumpCSVParserSuite) TestTSV(c *C) { types.NewStringDatum("0000-00-00"), }, Length: 14, - }) - c.Assert(parser, posEq, 52, 2) + }, parser.LastRow()) + assertPosEqual(t, parser, 52, 2) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 3, Row: []types.Datum{ types.NewStringDatum("0"), @@ -527,22 +508,22 @@ func (s *testMydumpCSVParserSuite) TestTSV(c *C) { types.NewStringDatum("1999-12-31"), }, Length: 23, - }) - c.Assert(parser, posEq, 80, 3) + }, parser.LastRow()) + assertPosEqual(t, parser, 80, 3) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestCsvWithWhiteSpaceLine(c *C) { +func TestCsvWithWhiteSpaceLine(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, } data := " \r\n\r\n0,,abc\r\n \r\n123,1999-12-31,test\r\n" - parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(data), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(data), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("0"), @@ -550,11 +531,11 @@ func (s *testMydumpCSVParserSuite) TestCsvWithWhiteSpaceLine(c *C) { types.NewStringDatum("abc"), }, Length: 4, - }) + }, parser.LastRow()) - c.Assert(parser, posEq, 12, 1) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + assertPosEqual(t, parser, 12, 1) + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewStringDatum("123"), @@ -562,16 +543,16 @@ func (s *testMydumpCSVParserSuite) TestCsvWithWhiteSpaceLine(c *C) { types.NewStringDatum("test"), }, Length: 17, - }) - c.Assert(parser.Close(), IsNil) + }, parser.LastRow()) + require.Nil(t, parser.Close()) cfg.Header = true data = " \r\na,b,c\r\n0,,abc\r\n" - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(data), int64(config.ReadBlockSize), s.ioWorkers, true, nil) - c.Assert(err, IsNil) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.Columns(), DeepEquals, []string{"a", "b", "c"}) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(data), int64(config.ReadBlockSize), ioWorkers, true, nil) + require.NoError(t, err) + require.Nil(t, parser.ReadRow()) + require.Equal(t, []string{"a", "b", "c"}, parser.Columns()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum("0"), @@ -579,84 +560,84 @@ func (s *testMydumpCSVParserSuite) TestCsvWithWhiteSpaceLine(c *C) { types.NewStringDatum("abc"), }, Length: 4, - }) + }, parser.LastRow()) - c.Assert(parser, posEq, 17, 1) - c.Assert(parser.Close(), IsNil) + assertPosEqual(t, parser, 17, 1) + require.Nil(t, parser.Close()) } -func (s *testMydumpCSVParserSuite) TestEmpty(c *C) { +func TestEmpty(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, } - parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(""), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(""), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) // Try again with headers. cfg.Header = true - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(""), int64(config.ReadBlockSize), s.ioWorkers, true, nil) - c.Assert(err, IsNil) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader(""), int64(config.ReadBlockSize), ioWorkers, true, nil) + require.NoError(t, err) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) - parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader("h\n"), int64(config.ReadBlockSize), s.ioWorkers, true, nil) - c.Assert(err, IsNil) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + parser, err = mydump.NewCSVParser(&cfg, mydump.NewStringReader("h\n"), int64(config.ReadBlockSize), ioWorkers, true, nil) + require.NoError(t, err) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestCRLF(c *C) { +func TestCRLF(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, } - parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader("a\rb\r\nc\n\n\n\nd"), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader("a\rb\r\nc\n\n\n\nd"), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{types.NewStringDatum("a")}, Length: 1, - }) + }, parser.LastRow()) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{types.NewStringDatum("b")}, Length: 1, - }) + }, parser.LastRow()) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 3, Row: []types.Datum{types.NewStringDatum("c")}, Length: 1, - }) + }, parser.LastRow()) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 4, Row: []types.Datum{types.NewStringDatum("d")}, Length: 1, - }) + }, parser.LastRow()) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestQuotedSeparator(c *C) { +func TestQuotedSeparator(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, } - parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(`",",','`), int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + parser, err := mydump.NewCSVParser(&cfg, mydump.NewStringReader(`",",','`), int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) + require.Nil(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewStringDatum(","), @@ -664,12 +645,12 @@ func (s *testMydumpCSVParserSuite) TestQuotedSeparator(c *C) { types.NewStringDatum("'"), }, Length: 3, - }) + }, parser.LastRow()) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpCSVParserSuite) TestConsecutiveFields(c *C) { +func TestConsecutiveFields(t *testing.T) { // Note: the behavior of reading `"xxx"yyy` here is undefined in RFC 4180. // Python's CSV module returns `xxxyyy`. // Rust's CSV package returns `xxxyyy`. @@ -691,15 +672,15 @@ func (s *testMydumpCSVParserSuite) TestConsecutiveFields(c *C) { `abc""`, } - s.runFailingTestCases(c, &cfg, int64(config.ReadBlockSize), testCases) + runFailingTestCasesCSV(t, &cfg, int64(config.ReadBlockSize), testCases) cfg.CSV.Delimiter = "|+|" - s.runFailingTestCases(c, &cfg, int64(config.ReadBlockSize), []string{ + runFailingTestCasesCSV(t, &cfg, int64(config.ReadBlockSize), []string{ "abc|1|+||+|\r\n", }) } -func (s *testMydumpCSVParserSuite) TestSpecialChars(c *C) { +func TestSpecialChars(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{Separator: ",", Delimiter: `"`}, } @@ -734,10 +715,10 @@ func (s *testMydumpCSVParserSuite) TestSpecialChars(c *C) { }, } - s.runTestCases(c, &cfg, int64(config.ReadBlockSize), testCases) + runTestCasesCSV(t, &cfg, int64(config.ReadBlockSize), testCases) } -func (s *testMydumpCSVParserSuite) TestContinuation(c *C) { +func TestContinuationCSV(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: ",", @@ -769,10 +750,10 @@ func (s *testMydumpCSVParserSuite) TestContinuation(c *C) { }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) } -func (s *testMydumpCSVParserSuite) TestBackslashAsSep(c *C) { +func TestBackslashAsSep(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: `\`, @@ -791,15 +772,15 @@ func (s *testMydumpCSVParserSuite) TestBackslashAsSep(c *C) { }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) failingInputs := []string{ `"\`, } - s.runFailingTestCases(c, &cfg, 1, failingInputs) + runFailingTestCasesCSV(t, &cfg, 1, failingInputs) } -func (s *testMydumpCSVParserSuite) TestBackslashAsDelim(c *C) { +func TestBackslashAsDelim(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: ",", @@ -813,12 +794,12 @@ func (s *testMydumpCSVParserSuite) TestBackslashAsDelim(c *C) { expected: [][]types.Datum{{nullDatum}}, }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) failingInputs := []string{ `"\`, } - s.runFailingTestCases(c, &cfg, 1, failingInputs) + runFailingTestCasesCSV(t, &cfg, 1, failingInputs) } // errorReader implements the Reader interface which always returns an error. @@ -836,19 +817,19 @@ func (*errorReader) Close() error { return errors.New("fake close error") } -func (s *testMydumpCSVParserSuite) TestReadError(c *C) { +func TestReadError(t *testing.T) { cfg := config.CSVConfig{ Separator: ",", Delimiter: `"`, } - parser, err := mydump.NewCSVParser(&cfg, &errorReader{}, int64(config.ReadBlockSize), s.ioWorkers, false, nil) - c.Assert(err, IsNil) - c.Assert(parser.ReadRow(), ErrorMatches, "fake read error") + parser, err := mydump.NewCSVParser(&cfg, &errorReader{}, int64(config.ReadBlockSize), ioWorkers, false, nil) + require.NoError(t, err) + require.Regexp(t, "fake read error", parser.ReadRow().Error()) } // TestSyntaxErrorLog checks that a syntax error won't dump huge strings into the log. -func (s *testMydumpCSVParserSuite) TestSyntaxErrorLog(c *C) { +func TestSyntaxErrorLog(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: "\t", @@ -857,21 +838,21 @@ func (s *testMydumpCSVParserSuite) TestSyntaxErrorLog(c *C) { } tc := mydump.NewStringReader("x'" + strings.Repeat("y", 50000)) - parser, err := mydump.NewCSVParser(&cfg.CSV, tc, 50000, s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err := mydump.NewCSVParser(&cfg.CSV, tc, 50000, ioWorkers, false, nil) + require.NoError(t, err) logger, buffer := log.MakeTestLogger() parser.SetLogger(logger) - c.Assert(parser.ReadRow(), ErrorMatches, "syntax error.*") - c.Assert(logger.Sync(), IsNil) + require.Regexp(t, "syntax error.*", parser.ReadRow().Error()) + require.Nil(t, logger.Sync()) - c.Assert( - buffer.Stripped(), Equals, + require.Equal(t, `{"$lvl":"ERROR","$msg":"syntax error","pos":2,"content":"`+strings.Repeat("y", 256)+`"}`, + buffer.Stripped(), ) } // TestTrimLastSep checks that set `TrimLastSep` to true trim only the last empty filed. -func (s *testMydumpCSVParserSuite) TestTrimLastSep(c *C) { +func TestTrimLastSep(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: ",", @@ -883,19 +864,19 @@ func (s *testMydumpCSVParserSuite) TestTrimLastSep(c *C) { &cfg.CSV, mydump.NewStringReader("123,456,789,\r\na,b,,\r\n,,,\r\n\"a\",\"\",\"\",\r\n"), int64(config.ReadBlockSize), - s.ioWorkers, + ioWorkers, false, nil, ) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 4; i++ { - c.Assert(parser.ReadRow(), IsNil) - c.Assert(len(parser.LastRow().Row), Equals, 3) + require.Nil(t, parser.ReadRow()) + require.Len(t, parser.LastRow().Row, 3) } } // TestTerminator checks for customized terminators. -func (s *testMydumpCSVParserSuite) TestTerminator(c *C) { +func TestTerminator(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: "|+|", @@ -913,7 +894,7 @@ func (s *testMydumpCSVParserSuite) TestTerminator(c *C) { }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) cfg.CSV.Delimiter = "|+>" @@ -926,10 +907,10 @@ func (s *testMydumpCSVParserSuite) TestTerminator(c *C) { }, }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) } -func (s *testMydumpCSVParserSuite) TestCharsetConversion(c *C) { +func TestCharsetConversion(t *testing.T) { cfg := config.MydumperRuntime{ CSV: config.CSVConfig{ Separator: ",", @@ -939,14 +920,14 @@ func (s *testMydumpCSVParserSuite) TestCharsetConversion(c *C) { DataInvalidCharReplace: string(utf8.RuneError), } charsetConvertor, err := mydump.NewCharsetConvertor(cfg.DataCharacterSet, cfg.DataInvalidCharReplace) - c.Assert(err, IsNil) + require.NoError(t, err) originalInputPart1 := `不要温驯地走进那个良夜,老年应当在日暮时燃烧咆哮,怒斥,怒斥光明的消逝。 ` originalInputPart2 := `虽然智慧的人临终时懂得黑暗有理,因为他们的话没有迸发出闪电,他们也并不温驯地走进那个良夜。 ` // Insert an invalid char to test DataInvalidCharReplace. rawInput, err := charsetConvertor.Encode(originalInputPart1 + string([]byte{0x99}) + originalInputPart2) - c.Assert(err, IsNil) + require.NoError(t, err) testCases := []testCase{ { @@ -963,7 +944,7 @@ func (s *testMydumpCSVParserSuite) TestCharsetConversion(c *C) { }, } - s.runTestCases(c, &cfg, 1, testCases) + runTestCasesCSV(t, &cfg, 1, testCases) } // Run `go test github.com/pingcap/br/pkg/lightning/mydump -check.b -check.bmem -test.v` to get benchmark result. @@ -974,37 +955,35 @@ type benchCSVParserSuite struct { ioWorkers *worker.Pool } -var _ = Suite(&benchCSVParserSuite{}) - -func (s *benchCSVParserSuite) setupTest(c *C) { +func newBenchCSVParserSuite(b *testing.B) *benchCSVParserSuite { + var s benchCSVParserSuite s.ioWorkers = worker.NewPool(context.Background(), 5, "bench_csv") - - dir := c.MkDir() + dir := b.TempDir() s.csvPath = filepath.Join(dir, "input.csv") file, err := os.Create(s.csvPath) - c.Assert(err, IsNil) + require.NoError(b, err) defer func() { - c.Assert(file.Close(), IsNil) + require.NoError(b, file.Close()) }() - for i := 0; i < c.N; i++ { + for i := 0; i < b.N; i++ { _, err = file.WriteString("18,1,1,0.3650,GC,BARBARBAR,rw9AOV1AjoI1,50000.00,-10.00,10.00,1,1,djj3Q2XaIPoYVy1FuF,gc80Q2o82Au3C9xv,PYOolSxG3w,DI,265111111,7586538936787184,2020-02-26 20:06:00.193,OE,YCkSPBVqoJ2V5F8zWs87V5XzbaIY70aWCD4dgcB6bjUzCr5wOJCJ2TYH49J7yWyysbudJIxlTAEWSJahY7hswLtTsqyjEkrlsN8iDMAa9Poj29miJ08tnn2G8mL64IlyywvnRGbLbyGvWDdrOSF42RyUFTWVyqlDWc6Gr5wyMPYgvweKemzFDVD3kro5JsmBmJY08EK54nQoyfo2sScyb34zcM9GFo9ZQTwloINfPYQKXQm32m0XvU7jiNmYpFTFJQjdqA825SEvQqMMefG2WG4jVu9UPdhdUjRsFRd0Gw7YPKByOlcuY0eKxT7sAzMKXx2000RR6dqHNXe47oVYd\n") - c.Assert(err, IsNil) + require.NoError(b, err) } - c.ResetTimer() + return &s } -func (s *benchCSVParserSuite) BenchmarkReadRowUsingMydumpCSVParser(c *C) { - s.setupTest(c) +func BenchmarkReadRowUsingMydumpCSVParser(b *testing.B) { + s := newBenchCSVParserSuite(b) file, err := os.Open(s.csvPath) - c.Assert(err, IsNil) + require.NoError(b, err) defer func() { - c.Assert(file.Close(), IsNil) + require.NoError(b, file.Close()) }() cfg := config.CSVConfig{Separator: ","} - parser, err := mydump.NewCSVParser(&cfg, file, 65536, s.ioWorkers, false, nil) - c.Assert(err, IsNil) + parser, err := mydump.NewCSVParser(&cfg, file, 65536, ioWorkers, false, nil) + require.NoError(b, err) parser.SetLogger(log.Logger{Logger: zap.NewNop()}) rowsCount := 0 @@ -1018,18 +997,18 @@ func (s *benchCSVParserSuite) BenchmarkReadRowUsingMydumpCSVParser(c *C) { if errors.Cause(err) == io.EOF { break } - c.Fatal(err) + b.Fatal(err) } - c.Assert(rowsCount, Equals, c.N) + require.Equal(b, b.N, rowsCount) } -func (s *benchCSVParserSuite) BenchmarkReadRowUsingEncodingCSV(c *C) { - s.setupTest(c) +func BenchmarkReadRowUsingEncodingCSV(b *testing.B) { + s := newBenchCSVParserSuite(b) file, err := os.Open(s.csvPath) - c.Assert(err, IsNil) + require.NoError(b, err) defer func() { - c.Assert(file.Close(), IsNil) + require.Nil(b, file.Close()) }() csvParser := csv.NewReader(file) @@ -1050,7 +1029,7 @@ func (s *benchCSVParserSuite) BenchmarkReadRowUsingEncodingCSV(c *C) { if errors.Cause(err) == io.EOF { break } - c.Fatal(err) + b.Fatal(err) } - c.Assert(rowsCount, Equals, c.N) + require.Equal(b, b.N, rowsCount) } diff --git a/br/pkg/lightning/mydump/loader_test.go b/br/pkg/lightning/mydump/loader_test.go index 88cf8b3fa03b2..d694ce1e01674 100644 --- a/br/pkg/lightning/mydump/loader_test.go +++ b/br/pkg/lightning/mydump/loader_test.go @@ -20,28 +20,20 @@ import ( "path/filepath" "testing" - . "github.com/pingcap/check" filter "github.com/pingcap/tidb-tools/pkg/table-filter" router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/br/pkg/lightning/config" md "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testMydumpLoaderSuite{}) - -func TestMydumps(t *testing.T) { - TestingT(t) -} - type testMydumpLoaderSuite struct { cfg *config.Config sourceDir string } -func (s *testMydumpLoaderSuite) SetUpSuite(c *C) {} -func (s *testMydumpLoaderSuite) TearDownSuite(c *C) {} - func newConfigWithSourceDir(sourceDir string) *config.Config { path, _ := filepath.Abs(sourceDir) return &config.Config{ @@ -53,42 +45,46 @@ func newConfigWithSourceDir(sourceDir string) *config.Config { } } -func (s *testMydumpLoaderSuite) SetUpTest(c *C) { - s.sourceDir = c.MkDir() +func newTestMydumpLoaderSuite(t *testing.T) *testMydumpLoaderSuite { + var s testMydumpLoaderSuite + var err error + s.sourceDir = t.TempDir() + require.Nil(t, err) s.cfg = newConfigWithSourceDir(s.sourceDir) + return &s } -func (s *testMydumpLoaderSuite) touch(c *C, filename ...string) { +func (s *testMydumpLoaderSuite) touch(t *testing.T, filename ...string) { components := make([]string, len(filename)+1) components = append(components, s.sourceDir) components = append(components, filename...) path := filepath.Join(components...) err := os.WriteFile(path, nil, 0o644) - c.Assert(err, IsNil) + require.Nil(t, err) } -func (s *testMydumpLoaderSuite) mkdir(c *C, dirname string) { +func (s *testMydumpLoaderSuite) mkdir(t *testing.T, dirname string) { path := filepath.Join(s.sourceDir, dirname) err := os.Mkdir(path, 0o755) - c.Assert(err, IsNil) + require.Nil(t, err) } -func (s *testMydumpLoaderSuite) TestLoader(c *C) { +func TestLoader(t *testing.T) { ctx := context.Background() cfg := newConfigWithSourceDir("./not-exists") _, err := md.NewMyDumpLoader(ctx, cfg) // will check schema in tidb and data file later in DataCheck. - c.Assert(err, IsNil) + require.NoError(t, err) cfg = newConfigWithSourceDir("./examples") mdl, err := md.NewMyDumpLoader(ctx, cfg) - c.Assert(err, IsNil) + require.NoError(t, err) dbMetas := mdl.GetDatabases() - c.Assert(len(dbMetas), Equals, 1) + require.Len(t, dbMetas, 1) dbMeta := dbMetas[0] - c.Assert(dbMeta.Name, Equals, "mocker_test") - c.Assert(len(dbMeta.Tables), Equals, 4) + require.Equal(t, "mocker_test", dbMeta.Name) + require.Len(t, dbMeta.Tables, 4) expected := []struct { name string @@ -101,18 +97,19 @@ func (s *testMydumpLoaderSuite) TestLoader(c *C) { } for i, table := range expected { - c.Assert(dbMeta.Tables[i].Name, Equals, table.name) - c.Assert(len(dbMeta.Tables[i].DataFiles), Equals, table.dataFiles) + assert.Equal(t, table.name, dbMeta.Tables[i].Name) + assert.Equal(t, table.dataFiles, len(dbMeta.Tables[i].DataFiles)) } } -func (s *testMydumpLoaderSuite) TestEmptyDB(c *C) { +func TestEmptyDB(t *testing.T) { + s := newTestMydumpLoaderSuite(t) _, err := md.NewMyDumpLoader(context.Background(), s.cfg) // will check schema in tidb and data file later in DataCheck. - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testMydumpLoaderSuite) TestDuplicatedDB(c *C) { +func TestDuplicatedDB(t *testing.T) { /* Path/ a/ @@ -120,33 +117,35 @@ func (s *testMydumpLoaderSuite) TestDuplicatedDB(c *C) { b/ db-schema-create.sql */ - s.mkdir(c, "a") - s.touch(c, "a", "db-schema-create.sql") - s.mkdir(c, "b") - s.touch(c, "b", "db-schema-create.sql") + s := newTestMydumpLoaderSuite(t) + s.mkdir(t, "a") + s.touch(t, "a", "db-schema-create.sql") + s.mkdir(t, "b") + s.touch(t, "b", "db-schema-create.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, ErrorMatches, `invalid database schema file, duplicated item - .*[/\\]db-schema-create\.sql`) + require.Regexp(t, `invalid database schema file, duplicated item - .*[/\\]db-schema-create\.sql`, err) } -func (s *testMydumpLoaderSuite) TestTableNoHostDB(c *C) { +func TestTableNoHostDB(t *testing.T) { /* Path/ notdb-schema-create.sql db.tbl-schema.sql */ + s := newTestMydumpLoaderSuite(t) dir := s.sourceDir err := os.WriteFile(filepath.Join(dir, "notdb-schema-create.sql"), nil, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) err = os.WriteFile(filepath.Join(dir, "db.tbl-schema.sql"), nil, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testMydumpLoaderSuite) TestDuplicatedTable(c *C) { +func TestDuplicatedTable(t *testing.T) { /* Path/ db-schema-create.sql @@ -155,126 +154,137 @@ func (s *testMydumpLoaderSuite) TestDuplicatedTable(c *C) { b/ db.tbl-schema.sql */ + s := newTestMydumpLoaderSuite(t) - s.touch(c, "db-schema-create.sql") - s.mkdir(c, "a") - s.touch(c, "a", "db.tbl-schema.sql") - s.mkdir(c, "b") - s.touch(c, "b", "db.tbl-schema.sql") + s.touch(t, "db-schema-create.sql") + s.mkdir(t, "a") + s.touch(t, "a", "db.tbl-schema.sql") + s.mkdir(t, "b") + s.touch(t, "b", "db.tbl-schema.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, ErrorMatches, `invalid table schema file, duplicated item - .*db\.tbl-schema\.sql`) + require.Regexp(t, `invalid table schema file, duplicated item - .*db\.tbl-schema\.sql`, err) } -func (s *testMydumpLoaderSuite) TestTableInfoNotFound(c *C) { +func TestTableInfoNotFound(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + s.cfg.Mydumper.CharacterSet = "auto" - s.touch(c, "db-schema-create.sql") - s.touch(c, "db.tbl-schema.sql") + s.touch(t, "db-schema-create.sql") + s.touch(t, "db.tbl-schema.sql") ctx := context.Background() store, err := storage.NewLocalStorage(s.sourceDir) - c.Assert(err, IsNil) + require.NoError(t, err) loader, err := md.NewMyDumpLoader(ctx, s.cfg) - c.Assert(err, IsNil) + require.NoError(t, err) for _, dbMeta := range loader.GetDatabases() { dbSQL, err := dbMeta.GetSchema(ctx, store) - c.Assert(err, IsNil) - c.Assert(dbSQL, Equals, "CREATE DATABASE IF NOT EXISTS `db`") + require.NoError(t, err) + require.Equal(t, "CREATE DATABASE IF NOT EXISTS `db`", dbSQL) for _, tblMeta := range dbMeta.Tables { sql, err := tblMeta.GetSchema(ctx, store) - c.Assert(sql, Equals, "") - c.Assert(err, IsNil) + require.Equal(t, "", sql) + require.NoError(t, err) } } } -func (s *testMydumpLoaderSuite) TestTableUnexpectedError(c *C) { - s.touch(c, "db-schema-create.sql") - s.touch(c, "db.tbl-schema.sql") +func TestTableUnexpectedError(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + s.touch(t, "db-schema-create.sql") + s.touch(t, "db.tbl-schema.sql") ctx := context.Background() store, err := storage.NewLocalStorage(s.sourceDir) - c.Assert(err, IsNil) + require.NoError(t, err) loader, err := md.NewMyDumpLoader(ctx, s.cfg) - c.Assert(err, IsNil) + require.NoError(t, err) for _, dbMeta := range loader.GetDatabases() { for _, tblMeta := range dbMeta.Tables { sql, err := tblMeta.GetSchema(ctx, store) - c.Assert(sql, Equals, "") - c.Assert(err, ErrorMatches, "failed to decode db.tbl-schema.sql as : Unsupported encoding ") + require.Equal(t, "", sql) + require.Contains(t, err.Error(), "failed to decode db.tbl-schema.sql as : Unsupported encoding ") } } } -func (s *testMydumpLoaderSuite) TestDataNoHostDB(c *C) { +func TestDataNoHostDB(t *testing.T) { /* Path/ notdb-schema-create.sql db.tbl.sql */ + s := newTestMydumpLoaderSuite(t) - s.touch(c, "notdb-schema-create.sql") - s.touch(c, "db.tbl.sql") + s.touch(t, "notdb-schema-create.sql") + s.touch(t, "db.tbl.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) // will check schema in tidb and data file later in DataCheck. - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testMydumpLoaderSuite) TestDataNoHostTable(c *C) { +func TestDataNoHostTable(t *testing.T) { /* Path/ db-schema-create.sql db.tbl.sql */ + s := newTestMydumpLoaderSuite(t) - s.touch(c, "db-schema-create.sql") - s.touch(c, "db.tbl.sql") + s.touch(t, "db-schema-create.sql") + s.touch(t, "db.tbl.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) // will check schema in tidb and data file later in DataCheck. - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testMydumpLoaderSuite) TestViewNoHostDB(c *C) { +func TestViewNoHostDB(t *testing.T) { /* Path/ notdb-schema-create.sql db.tbl-schema-view.sql */ - s.touch(c, "notdb-schema-create.sql") - s.touch(c, "db.tbl-schema-view.sql") + s := newTestMydumpLoaderSuite(t) + + s.touch(t, "notdb-schema-create.sql") + s.touch(t, "db.tbl-schema-view.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, ErrorMatches, `invalid view schema file, miss host table schema for view 'tbl'`) + require.Contains(t, err.Error(), `invalid view schema file, miss host table schema for view 'tbl'`) } -func (s *testMydumpLoaderSuite) TestViewNoHostTable(c *C) { +func TestViewNoHostTable(t *testing.T) { /* Path/ db-schema-create.sql db.tbl-schema-view.sql */ + s := newTestMydumpLoaderSuite(t) - s.touch(c, "db-schema-create.sql") - s.touch(c, "db.tbl-schema-view.sql") + s.touch(t, "db-schema-create.sql") + s.touch(t, "db.tbl-schema-view.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, ErrorMatches, `invalid view schema file, miss host table schema for view 'tbl'`) + require.Contains(t, err.Error(), `invalid view schema file, miss host table schema for view 'tbl'`) } -func (s *testMydumpLoaderSuite) TestDataWithoutSchema(c *C) { +func TestDataWithoutSchema(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + dir := s.sourceDir p := filepath.Join(dir, "db.tbl.sql") err := os.WriteFile(p, nil, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) - c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{{ + require.NoError(t, err) + require.Equal(t, []*md.MDDatabaseMeta{{ Name: "db", SchemaFile: md.FileInfo{ TableName: filter.Table{ @@ -291,25 +301,27 @@ func (s *testMydumpLoaderSuite) TestDataWithoutSchema(c *C) { IsRowOrdered: true, IndexRatio: 0.0, }}, - }}) + }}, mdl.GetDatabases()) } -func (s *testMydumpLoaderSuite) TestTablesWithDots(c *C) { - s.touch(c, "db-schema-create.sql") - s.touch(c, "db.tbl.with.dots-schema.sql") - s.touch(c, "db.tbl.with.dots.0001.sql") - s.touch(c, "db.0002-schema.sql") - s.touch(c, "db.0002.sql") +func TestTablesWithDots(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + + s.touch(t, "db-schema-create.sql") + s.touch(t, "db.tbl.with.dots-schema.sql") + s.touch(t, "db.tbl.with.dots.0001.sql") + s.touch(t, "db.0002-schema.sql") + s.touch(t, "db.0002.sql") // insert some tables with file name structures which we're going to ignore. - s.touch(c, "db.v-schema-trigger.sql") - s.touch(c, "db.v-schema-post.sql") - s.touch(c, "db.sql") - s.touch(c, "db-schema.sql") + s.touch(t, "db.v-schema-trigger.sql") + s.touch(t, "db.v-schema-post.sql") + s.touch(t, "db.sql") + s.touch(t, "db-schema.sql") mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) - c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{{ + require.NoError(t, err) + require.Equal(t, []*md.MDDatabaseMeta{{ Name: "db", SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "db", Name: ""}, FileMeta: md.SourceFileMeta{Path: "db-schema-create.sql", Type: md.SourceTypeSchemaSchema}}, Tables: []*md.MDTableMeta{ @@ -330,10 +342,11 @@ func (s *testMydumpLoaderSuite) TestTablesWithDots(c *C) { IndexRatio: 0.0, }, }, - }}) + }}, mdl.GetDatabases()) } -func (s *testMydumpLoaderSuite) TestRouter(c *C) { +func TestRouter(t *testing.T) { + s := newTestMydumpLoaderSuite(t) s.cfg.Routes = []*router.TableRule{ { SchemaPattern: "a*", @@ -376,33 +389,33 @@ func (s *testMydumpLoaderSuite) TestRouter(c *C) { e0.f0-schema-view.sql */ - s.touch(c, "a0-schema-create.sql") - s.touch(c, "a0.t0-schema.sql") - s.touch(c, "a0.t0.1.sql") - s.touch(c, "a0.t1-schema.sql") - s.touch(c, "a0.t1.1.sql") + s.touch(t, "a0-schema-create.sql") + s.touch(t, "a0.t0-schema.sql") + s.touch(t, "a0.t0.1.sql") + s.touch(t, "a0.t1-schema.sql") + s.touch(t, "a0.t1.1.sql") - s.touch(c, "a1-schema-create.sql") - s.touch(c, "a1.s1-schema.sql") - s.touch(c, "a1.s1.1.sql") - s.touch(c, "a1.t2-schema.sql") - s.touch(c, "a1.t2.1.sql") - s.touch(c, "a1.v1-schema.sql") - s.touch(c, "a1.v1-schema-view.sql") + s.touch(t, "a1-schema-create.sql") + s.touch(t, "a1.s1-schema.sql") + s.touch(t, "a1.s1.1.sql") + s.touch(t, "a1.t2-schema.sql") + s.touch(t, "a1.t2.1.sql") + s.touch(t, "a1.v1-schema.sql") + s.touch(t, "a1.v1-schema-view.sql") - s.touch(c, "c0-schema-create.sql") - s.touch(c, "c0.t3-schema.sql") - s.touch(c, "c0.t3.1.sql") + s.touch(t, "c0-schema-create.sql") + s.touch(t, "c0.t3-schema.sql") + s.touch(t, "c0.t3.1.sql") - s.touch(c, "d0-schema-create.sql") + s.touch(t, "d0-schema-create.sql") - s.touch(c, "e0-schema-create.sql") - s.touch(c, "e0.f0-schema.sql") - s.touch(c, "e0.f0-schema-view.sql") + s.touch(t, "e0-schema-create.sql") + s.touch(t, "e0.f0-schema.sql") + s.touch(t, "e0.f0-schema-view.sql") mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) - c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{ + require.NoError(t, err) + require.Equal(t, []*md.MDDatabaseMeta{ { Name: "a1", SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "a1", Name: ""}, FileMeta: md.SourceFileMeta{Path: "a1-schema-create.sql", Type: md.SourceTypeSchemaSchema}}, @@ -493,22 +506,26 @@ func (s *testMydumpLoaderSuite) TestRouter(c *C) { }, }, }, - }) + }, mdl.GetDatabases()) } -func (s *testMydumpLoaderSuite) TestBadRouterRule(c *C) { +func TestBadRouterRule(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + s.cfg.Routes = []*router.TableRule{{ SchemaPattern: "a*b", TargetSchema: "ab", }} - s.touch(c, "a1b-schema-create.sql") + s.touch(t, "a1b-schema-create.sql") _, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, ErrorMatches, `.*pattern a\*b not valid`) + require.Regexp(t, `.*pattern a\*b not valid`, err.Error()) } -func (s *testMydumpLoaderSuite) TestFileRouting(c *C) { +func TestFileRouting(t *testing.T) { + s := newTestMydumpLoaderSuite(t) + s.cfg.Mydumper.DefaultFileRules = false s.cfg.Mydumper.FileRouters = []*config.FileRouteRule{ { @@ -542,23 +559,23 @@ func (s *testMydumpLoaderSuite) TestFileRouting(c *C) { }, } - s.mkdir(c, "d1") - s.mkdir(c, "d2") - s.touch(c, "d1/schema.sql") - s.touch(c, "d1/test-table.sql") - s.touch(c, "d1/test0.sql") - s.touch(c, "d1/test1.sql") - s.touch(c, "d1/test2.001.sql") - s.touch(c, "d1/v1-table.sql") - s.touch(c, "d1/v1-view.sql") - s.touch(c, "d1/t1-schema-create.sql") - s.touch(c, "d2/schema.sql") - s.touch(c, "d2/abc-table.sql") - s.touch(c, "abc.1.sql") + s.mkdir(t, "d1") + s.mkdir(t, "d2") + s.touch(t, "d1/schema.sql") + s.touch(t, "d1/test-table.sql") + s.touch(t, "d1/test0.sql") + s.touch(t, "d1/test1.sql") + s.touch(t, "d1/test2.001.sql") + s.touch(t, "d1/v1-table.sql") + s.touch(t, "d1/v1-view.sql") + s.touch(t, "d1/t1-schema-create.sql") + s.touch(t, "d2/schema.sql") + s.touch(t, "d2/abc-table.sql") + s.touch(t, "abc.1.sql") mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) - c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{ + require.NoError(t, err) + require.Equal(t, []*md.MDDatabaseMeta{ { Name: "d1", SchemaFile: md.FileInfo{TableName: filter.Table{Schema: "d1", Name: ""}, FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("d1/schema.sql"), Type: md.SourceTypeSchemaSchema}}, @@ -629,10 +646,10 @@ func (s *testMydumpLoaderSuite) TestFileRouting(c *C) { }, }, }, - }) + }, mdl.GetDatabases()) } -func (s *testMydumpLoaderSuite) TestInputWithSpecialChars(c *C) { +func TestInputWithSpecialChars(t *testing.T) { /* Path/ test-schema-create.sql @@ -648,24 +665,25 @@ func (s *testMydumpLoaderSuite) TestInputWithSpecialChars(c *C) { db%22.t%2522-schema.sql db%22.t%2522.0.csv */ - - s.touch(c, "test-schema-create.sql") - s.touch(c, "test.t%22-schema.sql") - s.touch(c, "test.t%22.sql") - s.touch(c, "test.t%2522-schema.sql") - s.touch(c, "test.t%2522.csv") - s.touch(c, "test.t%gg-schema.sql") - s.touch(c, "test.t%gg.csv") - s.touch(c, "test.t+gg-schema.sql") - s.touch(c, "test.t+gg.csv") - - s.touch(c, "db%22-schema-create.sql") - s.touch(c, "db%22.t%2522-schema.sql") - s.touch(c, "db%22.t%2522.0.csv") + s := newTestMydumpLoaderSuite(t) + + s.touch(t, "test-schema-create.sql") + s.touch(t, "test.t%22-schema.sql") + s.touch(t, "test.t%22.sql") + s.touch(t, "test.t%2522-schema.sql") + s.touch(t, "test.t%2522.csv") + s.touch(t, "test.t%gg-schema.sql") + s.touch(t, "test.t%gg.csv") + s.touch(t, "test.t+gg-schema.sql") + s.touch(t, "test.t+gg.csv") + + s.touch(t, "db%22-schema-create.sql") + s.touch(t, "db%22.t%2522-schema.sql") + s.touch(t, "db%22.t%2522.0.csv") mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) - c.Assert(err, IsNil) - c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{ + require.NoError(t, err) + require.Equal(t, []*md.MDDatabaseMeta{ { Name: `db"`, SchemaFile: md.FileInfo{TableName: filter.Table{Schema: `db"`, Name: ""}, FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("db%22-schema-create.sql"), Type: md.SourceTypeSchemaSchema}}, @@ -738,5 +756,5 @@ func (s *testMydumpLoaderSuite) TestInputWithSpecialChars(c *C) { }, }, }, - }) + }, mdl.GetDatabases()) } diff --git a/br/pkg/kv/session_test.go b/br/pkg/lightning/mydump/main_test.go similarity index 59% rename from br/pkg/kv/session_test.go rename to br/pkg/lightning/mydump/main_test.go index 4c3ddf4015f28..cfed013442c98 100644 --- a/br/pkg/kv/session_test.go +++ b/br/pkg/lightning/mydump/main_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 PingCAP, Inc. +// Copyright 2022 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,17 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -package kv +package mydump import ( "testing" - "github.com/pingcap/tidb/parser/mysql" - "github.com/stretchr/testify/require" + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" ) -func TestSession(t *testing.T) { - session := newSession(&SessionOptions{SQLMode: mysql.ModeNone, Timestamp: 1234567890, RowFormatVersion: "1"}) - _, err := session.Txn(true) - require.NoError(t, err) +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), + } + goleak.VerifyTestMain(m, opts...) } diff --git a/br/pkg/lightning/mydump/parquet_parser_test.go b/br/pkg/lightning/mydump/parquet_parser_test.go index 2962e6cb1c5c7..0231c2b4e1e6d 100644 --- a/br/pkg/lightning/mydump/parquet_parser_test.go +++ b/br/pkg/lightning/mydump/parquet_parser_test.go @@ -5,82 +5,80 @@ import ( "io" "path/filepath" "strconv" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/xitongsys/parquet-go-source/local" writer2 "github.com/xitongsys/parquet-go/writer" ) -type testParquetParserSuite struct{} - -var _ = Suite(testParquetParserSuite{}) - -func (s testParquetParserSuite) TestParquetParser(c *C) { +func TestParquetParser(t *testing.T) { type Test struct { S string `parquet:"name=sS, type=UTF8, encoding=PLAIN_DICTIONARY"` A int32 `parquet:"name=a_A, type=INT32"` } - dir := c.MkDir() + dir := t.TempDir() // prepare data name := "test123.parquet" testPath := filepath.Join(dir, name) pf, err := local.NewLocalFileWriter(testPath) - c.Assert(err, IsNil) + require.NoError(t, err) test := &Test{} writer, err := writer2.NewParquetWriter(pf, test, 2) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 100; i++ { test.A = int32(i) test.S = strconv.Itoa(i) - c.Assert(writer.Write(test), IsNil) + require.NoError(t, writer.Write(test)) } - c.Assert(writer.WriteStop(), IsNil) - c.Assert(pf.Close(), IsNil) + require.NoError(t, writer.WriteStop()) + require.NoError(t, pf.Close()) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) r, err := store.Open(context.TODO(), name) - c.Assert(err, IsNil) + require.NoError(t, err) reader, err := NewParquetParser(context.TODO(), store, r, name) - c.Assert(err, IsNil) + require.NoError(t, err) defer reader.Close() - c.Assert(reader.Columns(), DeepEquals, []string{"ss", "a_a"}) + require.Equal(t, []string{"ss", "a_a"}, reader.Columns()) verifyRow := func(i int) { - c.Assert(reader.lastRow.RowID, Equals, int64(i+1)) - c.Assert(len(reader.lastRow.Row), Equals, 2) - c.Assert(reader.lastRow.Row[0], DeepEquals, types.NewCollationStringDatum(strconv.Itoa(i), "")) - c.Assert(reader.lastRow.Row[1], DeepEquals, types.NewIntDatum(int64(i))) + require.Equal(t, int64(i+1), reader.lastRow.RowID) + require.Len(t, reader.lastRow.Row, 2) + require.Equal(t, types.NewCollationStringDatum(strconv.Itoa(i), ""), reader.lastRow.Row[0]) + require.Equal(t, types.NewIntDatum(int64(i)), reader.lastRow.Row[1]) } // test read some rows for i := 0; i < 10; i++ { - c.Assert(reader.ReadRow(), IsNil) + require.NoError(t, reader.ReadRow()) verifyRow(i) } // test set pos to pos < curpos + batchReadRowSize - c.Assert(reader.SetPos(15, 15), IsNil) - c.Assert(reader.ReadRow(), IsNil) + require.NoError(t, reader.SetPos(15, 15)) + require.NoError(t, reader.ReadRow()) verifyRow(15) // test set pos to pos > curpos + batchReadRowSize - c.Assert(reader.SetPos(80, 80), IsNil) + require.NoError(t, reader.SetPos(80, 80)) for i := 80; i < 100; i++ { - c.Assert(reader.ReadRow(), IsNil) + require.NoError(t, reader.ReadRow()) verifyRow(i) } - c.Assert(reader.ReadRow(), Equals, io.EOF) + require.ErrorIs(t, reader.ReadRow(), io.EOF) } -func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { +func TestParquetVariousTypes(t *testing.T) { type Test struct { Date int32 `parquet:"name=date, type=DATE"` TimeMillis int32 `parquet:"name=timemillis, type=TIME_MILLIS"` @@ -94,15 +92,15 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { Decimal6 int32 `parquet:"name=decimal6, type=DECIMAL, scale=4, precision=4, basetype=INT32"` } - dir := c.MkDir() + dir := t.TempDir() // prepare data name := "test123.parquet" testPath := filepath.Join(dir, name) pf, err := local.NewLocalFileWriter(testPath) - c.Assert(err, IsNil) + require.NoError(t, err) test := &Test{} writer, err := writer2.NewParquetWriter(pf, test, 2) - c.Assert(err, IsNil) + require.NoError(t, err) v := &Test{ Date: 18564, // 2020-10-29 @@ -115,30 +113,30 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { Decimal3: 123456789012345678, // 1234567890123456.78 Decimal6: -1, // -0.0001 } - c.Assert(writer.Write(v), IsNil) - c.Assert(writer.WriteStop(), IsNil) - c.Assert(pf.Close(), IsNil) + require.NoError(t, writer.Write(v)) + require.NoError(t, writer.WriteStop()) + require.NoError(t, pf.Close()) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) r, err := store.Open(context.TODO(), name) - c.Assert(err, IsNil) + require.NoError(t, err) reader, err := NewParquetParser(context.TODO(), store, r, name) - c.Assert(err, IsNil) + require.NoError(t, err) defer reader.Close() - c.Assert(len(reader.columns), Equals, 9) + require.Len(t, reader.columns, 9) - c.Assert(reader.ReadRow(), IsNil) + require.NoError(t, reader.ReadRow()) rowValue := []string{ "2020-10-29", "17:26:15.123Z", "17:26:15.123456Z", "2020-10-29 09:27:52.356Z", "2020-10-29 09:27:52.356956Z", "-123456.78", "0.0456", "1234567890123456.78", "-0.0001", } row := reader.lastRow.Row - c.Assert(len(rowValue), Equals, len(row)) + require.Len(t, rowValue, len(row)) for i := 0; i < len(row); i++ { - c.Assert(row[i].Kind(), Equals, types.KindString) - c.Assert(rowValue[i], Equals, row[i].GetString()) + assert.Equal(t, types.KindString, row[i].Kind()) + assert.Equal(t, row[i].GetString(), rowValue[i]) } type TestDecimal struct { @@ -160,9 +158,9 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { testPath = filepath.Join(dir, fileName) pf, err = local.NewLocalFileWriter(testPath) td := &TestDecimal{} - c.Assert(err, IsNil) + require.NoError(t, err) writer, err = writer2.NewParquetWriter(pf, td, 2) - c.Assert(err, IsNil) + require.NoError(t, err) for i, testCase := range cases { val := testCase[0].(int32) td.Decimal1 = val @@ -171,19 +169,19 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { } else { td.DecimalRef = nil } - c.Assert(writer.Write(td), IsNil) + assert.NoError(t, writer.Write(td)) } - c.Assert(writer.WriteStop(), IsNil) - c.Assert(pf.Close(), IsNil) + require.NoError(t, writer.WriteStop()) + require.NoError(t, pf.Close()) r, err = store.Open(context.TODO(), fileName) - c.Assert(err, IsNil) + require.NoError(t, err) reader, err = NewParquetParser(context.TODO(), store, r, fileName) - c.Assert(err, IsNil) + require.NoError(t, err) defer reader.Close() for i, testCase := range cases { - c.Assert(reader.ReadRow(), IsNil) + assert.NoError(t, reader.ReadRow()) vals := []types.Datum{types.NewCollationStringDatum(testCase[1].(string), "")} if i%2 == 0 { vals = append(vals, vals[0]) @@ -192,25 +190,25 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { } // because we always reuse the datums in reader.lastRow.Row, so we can't directly // compare will `DeepEqual` here - c.Assert(len(reader.lastRow.Row), Equals, len(vals)) + assert.Len(t, reader.lastRow.Row, len(vals)) for i, val := range vals { - c.Assert(reader.lastRow.Row[i].Kind(), Equals, val.Kind()) - c.Assert(reader.lastRow.Row[i].GetValue(), Equals, val.GetValue()) + assert.Equal(t, val.Kind(), reader.lastRow.Row[i].Kind()) + assert.Equal(t, val.GetValue(), reader.lastRow.Row[i].GetValue()) } } } -func (s testParquetParserSuite) TestParquetAurora(c *C) { +func TestParquetAurora(t *testing.T) { store, err := storage.NewLocalStorage("examples") - c.Assert(err, IsNil) + require.NoError(t, err) fileName := "test.parquet" r, err := store.Open(context.TODO(), fileName) - c.Assert(err, IsNil) + require.NoError(t, err) parser, err := NewParquetParser(context.TODO(), store, r, fileName) - c.Assert(err, IsNil) + require.NoError(t, err) - c.Assert(parser.Columns(), DeepEquals, []string{"id", "val1", "val2", "d1", "d2", "d3", "d4", "d5", "d6"}) + require.Equal(t, []string{"id", "val1", "val2", "d1", "d2", "d3", "d4", "d5", "d6"}, parser.Columns()) expectedRes := [][]interface{}{ {int64(1), int64(1), "0", int64(123), "1.23", "0.00000001", "1234567890", "123", "1.23000000"}, @@ -238,21 +236,21 @@ func (s testParquetParserSuite) TestParquetAurora(c *C) { for i := 0; i < len(expectedRes); i++ { err = parser.ReadRow() - c.Assert(err, IsNil) + assert.NoError(t, err) expectedValues := expectedRes[i] row := parser.LastRow().Row - c.Assert(len(expectedValues), Equals, len(row)) + assert.Len(t, expectedValues, len(row)) for j := 0; j < len(row); j++ { switch v := expectedValues[j].(type) { case int64: - c.Assert(v, Equals, row[j].GetInt64()) + assert.Equal(t, row[j].GetInt64(), v) case string: - c.Assert(v, Equals, row[j].GetString()) + assert.Equal(t, row[j].GetString(), v) default: - c.Error("unexpected value: ", expectedValues[j]) + t.Fatal("unexpected value: ", expectedValues[j]) } } } - c.Assert(parser.ReadRow(), Equals, io.EOF) + require.ErrorIs(t, parser.ReadRow(), io.EOF) } diff --git a/br/pkg/lightning/mydump/parser_test.go b/br/pkg/lightning/mydump/parser_test.go index 53242022daae7..80dfcdc17c675 100644 --- a/br/pkg/lightning/mydump/parser_test.go +++ b/br/pkg/lightning/mydump/parser_test.go @@ -15,51 +15,41 @@ package mydump_test import ( - "context" + "fmt" "io" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/mydump" - "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testMydumpParserSuite{}) - -type testMydumpParserSuite struct { - ioWorkers *worker.Pool -} - -func (s *testMydumpParserSuite) SetUpSuite(c *C) { - s.ioWorkers = worker.NewPool(context.Background(), 5, "test_sql") -} -func (s *testMydumpParserSuite) TearDownSuite(c *C) {} - -func (s *testMydumpParserSuite) runTestCases(c *C, mode mysql.SQLMode, blockBufSize int64, cases []testCase) { +func runTestCases(t *testing.T, mode mysql.SQLMode, blockBufSize int64, cases []testCase) { for _, tc := range cases { - parser := mydump.NewChunkParser(mode, mydump.NewStringReader(tc.input), blockBufSize, s.ioWorkers) + parser := mydump.NewChunkParser(mode, mydump.NewStringReader(tc.input), blockBufSize, ioWorkers) for i, row := range tc.expected { e := parser.ReadRow() - comment := Commentf("input = %q, row = %d, err = %s", tc.input, i+1, errors.ErrorStack(e)) - c.Assert(e, IsNil, comment) - c.Assert(parser.LastRow().RowID, DeepEquals, int64(i)+1) - c.Assert(parser.LastRow().Row, DeepEquals, row) + comment := fmt.Sprintf("input = %q, row = %d, err = %s", tc.input, i+1, errors.ErrorStack(e)) + assert.NoError(t, e, comment) + assert.Equal(t, int64(i)+1, parser.LastRow().RowID, comment) + assert.Equal(t, row, parser.LastRow().Row, comment) } - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF, Commentf("input = %q", tc.input)) + assert.ErrorIsf(t, errors.Cause(parser.ReadRow()), io.EOF, "input = %q", tc.input) } } -func (s *testMydumpParserSuite) runFailingTestCases(c *C, mode mysql.SQLMode, blockBufSize int64, cases []string) { +func runFailingTestCases(t *testing.T, mode mysql.SQLMode, blockBufSize int64, cases []string) { for _, tc := range cases { - parser := mydump.NewChunkParser(mode, mydump.NewStringReader(tc), blockBufSize, s.ioWorkers) - c.Assert(parser.ReadRow(), ErrorMatches, "syntax error.*", Commentf("input = %q", tc)) + parser := mydump.NewChunkParser(mode, mydump.NewStringReader(tc), blockBufSize, ioWorkers) + assert.Regexpf(t, "syntax error.*", parser.ReadRow().Error(), "input = %q", tc) } } -func (s *testMydumpParserSuite) TestReadRow(c *C) { +func TestReadRow(t *testing.T) { reader := mydump.NewStringReader( "/* whatever pragmas */;" + "INSERT INTO `namespaced`.`table` (columns, more, columns) VALUES (1,-2, 3),\n(4,5., 6);" + @@ -67,10 +57,10 @@ func (s *testMydumpParserSuite) TestReadRow(c *C) { "insert another_table values (10,11e1,12, '(13)', '(', 14, ')');", ) - parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), s.ioWorkers) + parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), ioWorkers) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.NoError(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 1, Row: []types.Datum{ types.NewUintDatum(1), @@ -78,14 +68,14 @@ func (s *testMydumpParserSuite) TestReadRow(c *C) { types.NewUintDatum(3), }, Length: 62, - }) - c.Assert(parser.Columns(), DeepEquals, []string{"columns", "more", "columns"}) + }, parser.LastRow()) + require.Equal(t, []string{"columns", "more", "columns"}, parser.Columns()) offset, rowID := parser.Pos() - c.Assert(offset, Equals, int64(97)) - c.Assert(rowID, Equals, int64(1)) + require.Equal(t, int64(97), offset) + require.Equal(t, int64(1), rowID) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.NoError(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 2, Row: []types.Datum{ types.NewUintDatum(4), @@ -93,14 +83,14 @@ func (s *testMydumpParserSuite) TestReadRow(c *C) { types.NewUintDatum(6), }, Length: 6, - }) - c.Assert(parser.Columns(), DeepEquals, []string{"columns", "more", "columns"}) + }, parser.LastRow()) + require.Equal(t, []string{"columns", "more", "columns"}, parser.Columns()) offset, rowID = parser.Pos() - c.Assert(offset, Equals, int64(108)) - c.Assert(rowID, Equals, int64(2)) + require.Equal(t, int64(108), offset) + require.Equal(t, int64(2), rowID) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.NoError(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 3, Row: []types.Datum{ types.NewUintDatum(7), @@ -108,14 +98,14 @@ func (s *testMydumpParserSuite) TestReadRow(c *C) { types.NewUintDatum(9), }, Length: 42, - }) - c.Assert(parser.Columns(), DeepEquals, []string{"x", "y", "z"}) + }, parser.LastRow()) + require.Equal(t, []string{"x", "y", "z"}, parser.Columns()) offset, rowID = parser.Pos() - c.Assert(offset, Equals, int64(159)) - c.Assert(rowID, Equals, int64(3)) + require.Equal(t, int64(159), offset) + require.Equal(t, int64(3), rowID) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.LastRow(), DeepEquals, mydump.Row{ + require.NoError(t, parser.ReadRow()) + require.Equal(t, mydump.Row{ RowID: 4, Row: []types.Datum{ types.NewUintDatum(10), @@ -127,27 +117,27 @@ func (s *testMydumpParserSuite) TestReadRow(c *C) { types.NewStringDatum(")"), }, Length: 49, - }) - c.Assert(parser.Columns(), IsNil) + }, parser.LastRow()) + require.Nil(t, parser.Columns()) offset, rowID = parser.Pos() - c.Assert(offset, Equals, int64(222)) - c.Assert(rowID, Equals, int64(4)) + require.Equal(t, int64(222), offset) + require.Equal(t, int64(4), rowID) - c.Assert(errors.Cause(parser.ReadRow()), Equals, io.EOF) + require.ErrorIs(t, errors.Cause(parser.ReadRow()), io.EOF) } -func (s *testMydumpParserSuite) TestReadChunks(c *C) { +func TestReadChunks(t *testing.T) { reader := mydump.NewStringReader(` INSERT foo VALUES (1,2,3,4),(5,6,7,8),(9,10,11,12); INSERT foo VALUES (13,14,15,16),(17,18,19,20),(21,22,23,24),(25,26,27,28); INSERT foo VALUES (29,30,31,32),(33,34,35,36); `) - parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), s.ioWorkers) + parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), ioWorkers) chunks, err := mydump.ReadChunks(parser, 32) - c.Assert(err, IsNil) - c.Assert(chunks, DeepEquals, []mydump.Chunk{ + require.NoError(t, err) + require.Equal(t, []mydump.Chunk{ { Offset: 0, EndOffset: 40, @@ -178,10 +168,10 @@ func (s *testMydumpParserSuite) TestReadChunks(c *C) { PrevRowIDMax: 8, RowIDMax: 9, }, - }) + }, chunks) } -func (s *testMydumpParserSuite) TestNestedRow(c *C) { +func TestNestedRow(t *testing.T) { reader := mydump.NewStringReader(` INSERT INTO exam_detail VALUES ("123",CONVERT("{}" USING UTF8MB4)), @@ -189,11 +179,11 @@ func (s *testMydumpParserSuite) TestNestedRow(c *C) { ("789",CONVERT("[]" USING UTF8MB4)); `) - parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), s.ioWorkers) + parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), ioWorkers) chunks, err := mydump.ReadChunks(parser, 96) - c.Assert(err, IsNil) - c.Assert(chunks, DeepEquals, []mydump.Chunk{ + require.NoError(t, err) + require.Equal(t, []mydump.Chunk{ { Offset: 0, EndOffset: 117, @@ -206,10 +196,10 @@ func (s *testMydumpParserSuite) TestNestedRow(c *C) { PrevRowIDMax: 2, RowIDMax: 3, }, - }) + }, chunks) } -func (s *testMydumpParserSuite) TestVariousSyntax(c *C) { +func TestVariousSyntax(t *testing.T) { testCases := []testCase{ { input: "INSERT INTO foobar VALUES (1, 2);", @@ -356,10 +346,10 @@ func (s *testMydumpParserSuite) TestVariousSyntax(c *C) { }, } - s.runTestCases(c, mysql.ModeNone, int64(config.ReadBlockSize), testCases) + runTestCases(t, mysql.ModeNone, int64(config.ReadBlockSize), testCases) } -func (s *testMydumpParserSuite) TestContinuation(c *C) { +func TestContinuation(t *testing.T) { testCases := []testCase{ { input: ` @@ -377,10 +367,10 @@ func (s *testMydumpParserSuite) TestContinuation(c *C) { }, } - s.runTestCases(c, mysql.ModeNone, 1, testCases) + runTestCases(t, mysql.ModeNone, 1, testCases) } -func (s *testMydumpParserSuite) TestPseudoKeywords(c *C) { +func TestPseudoKeywords(t *testing.T) { reader := mydump.NewStringReader(` INSERT INTO t ( c, C, @@ -422,9 +412,9 @@ func (s *testMydumpParserSuite) TestPseudoKeywords(c *C) { ) VALUES (); `) - parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), s.ioWorkers) - c.Assert(parser.ReadRow(), IsNil) - c.Assert(parser.Columns(), DeepEquals, []string{ + parser := mydump.NewChunkParser(mysql.ModeNone, reader, int64(config.ReadBlockSize), ioWorkers) + require.NoError(t, parser.ReadRow()) + require.Equal(t, []string{ "c", "c", "co", "co", "con", "con", @@ -461,10 +451,10 @@ func (s *testMydumpParserSuite) TestPseudoKeywords(c *C) { "ins", "ins", "inse", "inse", "inser", "inser", - }) + }, parser.Columns()) } -func (s *testMydumpParserSuite) TestSyntaxError(c *C) { +func TestSyntaxError(t *testing.T) { inputs := []string{ "('xxx)", `("xxx)`, @@ -489,13 +479,13 @@ func (s *testMydumpParserSuite) TestSyntaxError(c *C) { "/* ...", } - s.runFailingTestCases(c, mysql.ModeNone, int64(config.ReadBlockSize), inputs) + runFailingTestCases(t, mysql.ModeNone, int64(config.ReadBlockSize), inputs) } // Various syntax error cases collected via fuzzing. // These cover most of the tokenizer branches. -func (s *testMydumpParserSuite) TestMoreSyntaxError(c *C) { +func TestMoreSyntaxError(t *testing.T) { inputs := []string{ " usin0", "- ", @@ -862,11 +852,11 @@ func (s *testMydumpParserSuite) TestMoreSyntaxError(c *C) { "x00`0`Valu0", } - s.runFailingTestCases(c, mysql.ModeNone, 1, inputs) - s.runFailingTestCases(c, mysql.ModeNoBackslashEscapes, 1, inputs) + runFailingTestCases(t, mysql.ModeNone, 1, inputs) + runFailingTestCases(t, mysql.ModeNoBackslashEscapes, 1, inputs) } -func (s *testMydumpParserSuite) TestMoreEmptyFiles(c *C) { +func TestMoreEmptyFiles(t *testing.T) { testCases := []testCase{ {input: ""}, {input: "--\t"}, @@ -884,6 +874,6 @@ func (s *testMydumpParserSuite) TestMoreEmptyFiles(c *C) { {input: "--\r"}, } - s.runTestCases(c, mysql.ModeNone, 1, testCases) - s.runTestCases(c, mysql.ModeNoBackslashEscapes, 1, testCases) + runTestCases(t, mysql.ModeNone, 1, testCases) + runTestCases(t, mysql.ModeNoBackslashEscapes, 1, testCases) } diff --git a/br/pkg/lightning/mydump/reader_test.go b/br/pkg/lightning/mydump/reader_test.go index 0ff37ad666747..e7506ea869782 100644 --- a/br/pkg/lightning/mydump/reader_test.go +++ b/br/pkg/lightning/mydump/reader_test.go @@ -19,45 +19,39 @@ import ( "errors" "os" "path/filepath" + "testing" "github.com/golang/mock/gomock" - . "github.com/pingcap/check" . "github.com/pingcap/tidb/br/pkg/lightning/mydump" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testMydumpReaderSuite{}) - -type testMydumpReaderSuite struct{} - -func (s *testMydumpReaderSuite) SetUpSuite(c *C) {} -func (s *testMydumpReaderSuite) TearDownSuite(c *C) {} - -func (s *testMydumpReaderSuite) TestExportStatementNoTrailingNewLine(c *C) { - dir := c.MkDir() +func TestExportStatementNoTrailingNewLine(t *testing.T) { + dir := t.TempDir() file, err := os.Create(filepath.Join(dir, "tidb_lightning_test_reader")) - c.Assert(err, IsNil) + require.NoError(t, err) defer os.Remove(file.Name()) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = file.Write([]byte("CREATE DATABASE whatever;")) - c.Assert(err, IsNil) + require.NoError(t, err) stat, err := file.Stat() - c.Assert(err, IsNil) + require.NoError(t, err) err = file.Close() - c.Assert(err, IsNil) + require.NoError(t, err) f := FileInfo{FileMeta: SourceFileMeta{Path: stat.Name(), FileSize: stat.Size()}} data, err := ExportStatement(context.TODO(), store, f, "auto") - c.Assert(err, IsNil) - c.Assert(data, DeepEquals, []byte("CREATE DATABASE whatever;")) + require.NoError(t, err) + require.Equal(t, []byte("CREATE DATABASE whatever;"), data) } -func (s *testMydumpReaderSuite) TestExportStatementWithComment(c *C) { - s.exportStatmentShouldBe(c, ` +func TestExportStatementWithComment(t *testing.T) { + exportStatmentShouldBe(t, ` /* whatever blabla multiple lines comment multiple lines comment @@ -69,8 +63,8 @@ func (s *testMydumpReaderSuite) TestExportStatementWithComment(c *C) { `, "CREATE DATABASE whatever;") } -func (s *testMydumpReaderSuite) TestExportStatementWithCommentNoTrailingNewLine(c *C) { - s.exportStatmentShouldBe(c, ` +func TestExportStatementWithCommentNoTrailingNewLine(t *testing.T) { + exportStatmentShouldBe(t, ` /* whatever blabla multiple lines comment multiple lines comment @@ -81,73 +75,73 @@ func (s *testMydumpReaderSuite) TestExportStatementWithCommentNoTrailingNewLine( CREATE DATABASE whatever;`, "CREATE DATABASE whatever;") } -func (s *testMydumpReaderSuite) exportStatmentShouldBe(c *C, stmt string, expected string) { - dir := c.MkDir() +func exportStatmentShouldBe(t *testing.T, stmt string, expected string) { + dir := t.TempDir() file, err := os.Create(filepath.Join(dir, "tidb_lightning_test_reader")) - c.Assert(err, IsNil) + require.NoError(t, err) defer os.Remove(file.Name()) _, err = file.Write([]byte(stmt)) - c.Assert(err, IsNil) + require.NoError(t, err) stat, err := file.Stat() - c.Assert(err, IsNil) + require.NoError(t, err) err = file.Close() - c.Assert(err, IsNil) + require.NoError(t, err) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) f := FileInfo{FileMeta: SourceFileMeta{Path: stat.Name(), FileSize: stat.Size()}} data, err := ExportStatement(context.TODO(), store, f, "auto") - c.Assert(err, IsNil) - c.Assert(data, DeepEquals, []byte(expected)) + require.NoError(t, err) + require.Equal(t, []byte(expected), data) } -func (s *testMydumpReaderSuite) TestExportStatementGBK(c *C) { - dir := c.MkDir() +func TestExportStatementGBK(t *testing.T) { + dir := t.TempDir() file, err := os.Create(filepath.Join(dir, "tidb_lightning_test_reader")) - c.Assert(err, IsNil) + require.NoError(t, err) defer os.Remove(file.Name()) _, err = file.Write([]byte("CREATE TABLE a (b int(11) COMMENT '")) - c.Assert(err, IsNil) + require.NoError(t, err) // "D7 DC B0 B8 C0 FD" is the GBK encoding of "总案例". _, err = file.Write([]byte{0xD7, 0xDC, 0xB0, 0xB8, 0xC0, 0xFD}) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = file.Write([]byte("');\n")) - c.Assert(err, IsNil) + require.NoError(t, err) stat, err := file.Stat() - c.Assert(err, IsNil) + require.NoError(t, err) err = file.Close() - c.Assert(err, IsNil) + require.NoError(t, err) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) f := FileInfo{FileMeta: SourceFileMeta{Path: stat.Name(), FileSize: stat.Size()}} data, err := ExportStatement(context.TODO(), store, f, "auto") - c.Assert(err, IsNil) - c.Assert(data, DeepEquals, []byte("CREATE TABLE a (b int(11) COMMENT '总案例');")) + require.NoError(t, err) + require.Equal(t, []byte("CREATE TABLE a (b int(11) COMMENT '总案例');"), data) } -func (s *testMydumpReaderSuite) TestExportStatementGibberishError(c *C) { - dir := c.MkDir() +func TestExportStatementGibberishError(t *testing.T) { + dir := t.TempDir() file, err := os.Create(filepath.Join(dir, "tidb_lightning_test_reader")) - c.Assert(err, IsNil) + require.NoError(t, err) defer os.Remove(file.Name()) _, err = file.Write([]byte("\x9e\x02\xdc\xfbZ/=n\xf3\xf2N8\xc1\xf2\xe9\xaa\xd0\x85\xc5}\x97\x07\xae6\x97\x99\x9c\x08\xcb\xe8;")) - c.Assert(err, IsNil) + require.NoError(t, err) stat, err := file.Stat() - c.Assert(err, IsNil) + require.NoError(t, err) err = file.Close() - c.Assert(err, IsNil) + require.NoError(t, err) store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) f := FileInfo{FileMeta: SourceFileMeta{Path: stat.Name(), FileSize: stat.Size()}} data, err := ExportStatement(context.TODO(), store, f, "auto") - c.Assert(data, HasLen, 0) - c.Assert(err, ErrorMatches, `failed to decode \w* as auto: invalid schema encoding`) + require.Len(t, data, 0) + require.Regexp(t, `failed to decode \w* as auto: invalid schema encoding`, err.Error()) } type AlwaysErrorReadSeekCloser struct{} @@ -164,8 +158,8 @@ func (AlwaysErrorReadSeekCloser) Close() error { return nil } -func (s *testMydumpReaderSuite) TestExportStatementHandleNonEOFError(c *C) { - controller := gomock.NewController(c) +func TestExportStatementHandleNonEOFError(t *testing.T) { + controller := gomock.NewController(t) defer controller.Finish() ctx := context.TODO() @@ -177,5 +171,5 @@ func (s *testMydumpReaderSuite) TestExportStatementHandleNonEOFError(c *C) { f := FileInfo{FileMeta: SourceFileMeta{Path: "no-perm-file", FileSize: 1}} _, err := ExportStatement(ctx, mockStorage, f, "auto") - c.Assert(err, ErrorMatches, "read error") + require.Contains(t, err.Error(), "read error") } diff --git a/br/pkg/lightning/mydump/region_test.go b/br/pkg/lightning/mydump/region_test.go index 6ee26692d4e8c..a1dbb9f290a69 100644 --- a/br/pkg/lightning/mydump/region_test.go +++ b/br/pkg/lightning/mydump/region_test.go @@ -18,21 +18,16 @@ import ( "context" "os" "path/filepath" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/config" . "github.com/pingcap/tidb/br/pkg/lightning/mydump" "github.com/pingcap/tidb/br/pkg/lightning/worker" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testMydumpRegionSuite{}) - -type testMydumpRegionSuite struct{} - -func (s *testMydumpRegionSuite) SetUpSuite(c *C) {} -func (s *testMydumpRegionSuite) TearDownSuite(c *C) {} - // var expectedTuplesCount = map[string]int64{ // "i": 1, // "report_case_high_risk": 1, @@ -43,7 +38,7 @@ func (s *testMydumpRegionSuite) TearDownSuite(c *C) {} /* TODO : test with specified 'regionBlockSize' ... */ -func (s *testMydumpRegionSuite) TestTableRegion(c *C) { +func TestTableRegion(t *testing.T) { cfg := newConfigWithSourceDir("./examples") loader, _ := NewMyDumpLoader(context.Background(), cfg) dbMeta := loader.GetDatabases()[0] @@ -51,7 +46,7 @@ func (s *testMydumpRegionSuite) TestTableRegion(c *C) { ioWorkers := worker.NewPool(context.Background(), 1, "io") for _, meta := range dbMeta.Tables { regions, err := MakeTableRegions(context.Background(), meta, 1, cfg, ioWorkers, loader.GetStore()) - c.Assert(err, IsNil) + require.NoError(t, err) // check - region-size vs file-size var tolFileSize int64 = 0 @@ -62,7 +57,7 @@ func (s *testMydumpRegionSuite) TestTableRegion(c *C) { for _, region := range regions { tolRegionSize += region.Size() } - c.Assert(tolRegionSize, Equals, tolFileSize) + require.Equal(t, tolFileSize, tolRegionSize) // // check - rows num // var tolRows int64 = 0 @@ -77,18 +72,18 @@ func (s *testMydumpRegionSuite) TestTableRegion(c *C) { for i := 1; i < regionNum; i++ { reg := regions[i] if preReg.FileMeta.Path == reg.FileMeta.Path { - c.Assert(reg.Offset(), Equals, preReg.Offset()+preReg.Size()) - c.Assert(reg.RowIDMin(), Equals, preReg.RowIDMin()+preReg.Rows()) + require.Equal(t, preReg.Offset()+preReg.Size(), reg.Offset()) + require.Equal(t, preReg.RowIDMin()+preReg.Rows(), reg.RowIDMin()) } else { - c.Assert(reg.Offset, Equals, 0) - c.Assert(reg.RowIDMin(), Equals, 1) + require.Equal(t, 0, reg.Offset()) + require.Equal(t, 1, reg.RowIDMin()) } preReg = reg } } } -func (s *testMydumpRegionSuite) TestAllocateEngineIDs(c *C) { +func TestAllocateEngineIDs(t *testing.T) { dataFileSizes := make([]float64, 700) for i := range dataFileSizes { dataFileSizes[i] = 1.0 @@ -103,7 +98,7 @@ func (s *testMydumpRegionSuite) TestAllocateEngineIDs(c *C) { for _, region := range filesRegions { actual[region.EngineID]++ } - c.Assert(actual, DeepEquals, expected, Commentf("%s", what)) + require.Equal(t, expected, actual, what) } // Batch size > Total size => Everything in the zero batch. @@ -169,7 +164,7 @@ func (s *testMydumpRegionSuite) TestAllocateEngineIDs(c *C) { }) } -func (s *testMydumpRegionSuite) TestSplitLargeFile(c *C) { +func TestSplitLargeFile(t *testing.T) { meta := &MDTableMeta{ DB: "csv", Name: "large_csv_file", @@ -192,7 +187,7 @@ func (s *testMydumpRegionSuite) TestSplitLargeFile(c *C) { } filePath := "./csv/split_large_file.csv" dataFileInfo, err := os.Stat(filePath) - c.Assert(err, IsNil) + require.NoError(t, err) fileSize := dataFileInfo.Size() fileInfo := FileInfo{FileMeta: SourceFileMeta{Path: filePath, Type: SourceTypeCSV, FileSize: fileSize}} colCnt := int64(3) @@ -214,20 +209,20 @@ func (s *testMydumpRegionSuite) TestSplitLargeFile(c *C) { ioWorker := worker.NewPool(context.Background(), 4, "io") store, err := storage.NewLocalStorage(".") - c.Assert(err, IsNil) + assert.NoError(t, err) _, regions, _, err := SplitLargeFile(context.Background(), meta, cfg, fileInfo, colCnt, prevRowIdxMax, ioWorker, store) - c.Assert(err, IsNil) - c.Assert(regions, HasLen, len(tc.offsets)) + assert.NoError(t, err) + assert.Len(t, regions, len(tc.offsets)) for i := range tc.offsets { - c.Assert(regions[i].Chunk.Offset, Equals, tc.offsets[i][0]) - c.Assert(regions[i].Chunk.EndOffset, Equals, tc.offsets[i][1]) - c.Assert(regions[i].Chunk.Columns, DeepEquals, columns) + assert.Equal(t, tc.offsets[i][0], regions[i].Chunk.Offset) + assert.Equal(t, tc.offsets[i][1], regions[i].Chunk.EndOffset) + assert.Equal(t, columns, regions[i].Chunk.Columns) } } } -func (s *testMydumpRegionSuite) TestSplitLargeFileNoNewLineAtEOF(c *C) { +func TestSplitLargeFileNoNewLineAtEOF(t *testing.T) { meta := &MDTableMeta{ DB: "csv", Name: "large_csv_file", @@ -250,17 +245,17 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileNoNewLineAtEOF(c *C) { }, } - dir := c.MkDir() + dir := t.TempDir() fileName := "test.csv" filePath := filepath.Join(dir, fileName) content := []byte("a,b\r\n123,456\r\n789,101") err := os.WriteFile(filePath, content, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) dataFileInfo, err := os.Stat(filePath) - c.Assert(err, IsNil) + require.NoError(t, err) fileSize := dataFileInfo.Size() fileInfo := FileInfo{FileMeta: SourceFileMeta{Path: fileName, Type: SourceTypeCSV, FileSize: fileSize}} colCnt := int64(2) @@ -269,21 +264,21 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileNoNewLineAtEOF(c *C) { ioWorker := worker.NewPool(context.Background(), 4, "io") store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) offsets := [][]int64{{4, 13}, {13, 21}} _, regions, _, err := SplitLargeFile(context.Background(), meta, cfg, fileInfo, colCnt, prevRowIdxMax, ioWorker, store) - c.Assert(err, IsNil) - c.Assert(regions, HasLen, len(offsets)) + require.NoError(t, err) + require.Len(t, regions, len(offsets)) for i := range offsets { - c.Assert(regions[i].Chunk.Offset, Equals, offsets[i][0]) - c.Assert(regions[i].Chunk.EndOffset, Equals, offsets[i][1]) - c.Assert(regions[i].Chunk.Columns, DeepEquals, columns) + require.Equal(t, offsets[i][0], regions[i].Chunk.Offset) + require.Equal(t, offsets[i][1], regions[i].Chunk.EndOffset) + require.Equal(t, columns, regions[i].Chunk.Columns) } } -func (s *testMydumpRegionSuite) TestSplitLargeFileWithCustomTerminator(c *C) { +func TestSplitLargeFileWithCustomTerminator(t *testing.T) { meta := &MDTableMeta{ DB: "csv", Name: "large_csv_with_custom_terminator", @@ -301,17 +296,17 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileWithCustomTerminator(c *C) { }, } - dir := c.MkDir() + dir := t.TempDir() fileName := "test2.csv" filePath := filepath.Join(dir, fileName) content := []byte("5|+|abc\ndef\nghi|+|6|+|\n7|+|xyz|+|8|+|\n9|+||+|10") err := os.WriteFile(filePath, content, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) dataFileInfo, err := os.Stat(filePath) - c.Assert(err, IsNil) + require.NoError(t, err) fileSize := dataFileInfo.Size() fileInfo := FileInfo{FileMeta: SourceFileMeta{Path: fileName, Type: SourceTypeCSV, FileSize: fileSize}} colCnt := int64(3) @@ -319,20 +314,20 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileWithCustomTerminator(c *C) { ioWorker := worker.NewPool(context.Background(), 4, "io") store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) offsets := [][]int64{{0, 23}, {23, 38}, {38, 47}} _, regions, _, err := SplitLargeFile(context.Background(), meta, cfg, fileInfo, colCnt, prevRowIdxMax, ioWorker, store) - c.Assert(err, IsNil) - c.Assert(regions, HasLen, len(offsets)) + require.NoError(t, err) + require.Len(t, regions, len(offsets)) for i := range offsets { - c.Assert(regions[i].Chunk.Offset, Equals, offsets[i][0]) - c.Assert(regions[i].Chunk.EndOffset, Equals, offsets[i][1]) + require.Equal(t, offsets[i][0], regions[i].Chunk.Offset) + require.Equal(t, offsets[i][1], regions[i].Chunk.EndOffset) } } -func (s *testMydumpRegionSuite) TestSplitLargeFileOnlyOneChunk(c *C) { +func TestSplitLargeFileOnlyOneChunk(t *testing.T) { meta := &MDTableMeta{ DB: "csv", Name: "large_csv_file", @@ -355,17 +350,17 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileOnlyOneChunk(c *C) { }, } - dir := c.MkDir() + dir := t.TempDir() fileName := "test.csv" filePath := filepath.Join(dir, fileName) content := []byte("field1,field2\r\n123,456\r\n") err := os.WriteFile(filePath, content, 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) dataFileInfo, err := os.Stat(filePath) - c.Assert(err, IsNil) + require.NoError(t, err) fileSize := dataFileInfo.Size() fileInfo := FileInfo{FileMeta: SourceFileMeta{Path: fileName, Type: SourceTypeCSV, FileSize: fileSize}} colCnt := int64(2) @@ -374,16 +369,16 @@ func (s *testMydumpRegionSuite) TestSplitLargeFileOnlyOneChunk(c *C) { ioWorker := worker.NewPool(context.Background(), 4, "io") store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) offsets := [][]int64{{14, 24}} _, regions, _, err := SplitLargeFile(context.Background(), meta, cfg, fileInfo, colCnt, prevRowIdxMax, ioWorker, store) - c.Assert(err, IsNil) - c.Assert(regions, HasLen, len(offsets)) + require.NoError(t, err) + require.Len(t, regions, len(offsets)) for i := range offsets { - c.Assert(regions[i].Chunk.Offset, Equals, offsets[i][0]) - c.Assert(regions[i].Chunk.EndOffset, Equals, offsets[i][1]) - c.Assert(regions[i].Chunk.Columns, DeepEquals, columns) + require.Equal(t, offsets[i][0], regions[i].Chunk.Offset) + require.Equal(t, offsets[i][1], regions[i].Chunk.EndOffset) + require.Equal(t, columns, regions[i].Chunk.Columns) } } diff --git a/br/pkg/lightning/mydump/router_test.go b/br/pkg/lightning/mydump/router_test.go index bbb504ef98b77..9e4a4d7cd3f28 100644 --- a/br/pkg/lightning/mydump/router_test.go +++ b/br/pkg/lightning/mydump/router_test.go @@ -2,17 +2,15 @@ package mydump import ( "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testFileRouterSuite{}) - -type testFileRouterSuite struct{} - -func (t *testFileRouterSuite) TestRouteParser(c *C) { +func TestRouteParser(t *testing.T) { // valid rules rules := []*config.FileRouteRule{ {Pattern: `^(?:[^/]*/)*([^/.]+)\.([^./]+)(?:\.[0-9]+)?\.(csv|sql)`, Schema: "$1", Table: "$2", Type: "$3"}, @@ -24,7 +22,7 @@ func (t *testFileRouterSuite) TestRouteParser(c *C) { } for _, r := range rules { _, err := NewFileRouter([]*config.FileRouteRule{r}) - c.Assert(err, IsNil) + assert.NoError(t, err) } // invalid rules @@ -35,48 +33,48 @@ func (t *testFileRouterSuite) TestRouteParser(c *C) { } for _, r := range invalidRules { _, err := NewFileRouter([]*config.FileRouteRule{r}) - c.Assert(err, NotNil) + assert.Error(t, err) } } -func (t *testFileRouterSuite) TestInvalidRouteRule(c *C) { +func TestInvalidRouteRule(t *testing.T) { rule := &config.FileRouteRule{} rules := []*config.FileRouteRule{rule} _, err := NewFileRouter(rules) - c.Assert(err, ErrorMatches, "`path` and `pattern` must not be both empty in \\[\\[mydumper.files\\]\\]") + require.Regexp(t, "`path` and `pattern` must not be both empty in \\[\\[mydumper.files\\]\\]", err.Error()) rule.Pattern = `^(?:[^/]*/)*([^/.]+)\.(?P[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$` _, err = NewFileRouter(rules) - c.Assert(err, ErrorMatches, "field 'type' match pattern can't be empty") + require.Regexp(t, "field 'type' match pattern can't be empty", err.Error()) rule.Type = "$type" _, err = NewFileRouter(rules) - c.Assert(err, ErrorMatches, "field 'schema' match pattern can't be empty") + require.Regexp(t, "field 'schema' match pattern can't be empty", err.Error()) rule.Schema = "$schema" _, err = NewFileRouter(rules) - c.Assert(err, ErrorMatches, "invalid named capture '\\$schema'") + require.Regexp(t, "invalid named capture '\\$schema'", err.Error()) rule.Schema = "$1" _, err = NewFileRouter(rules) - c.Assert(err, ErrorMatches, "field 'table' match pattern can't be empty") + require.Regexp(t, "field 'table' match pattern can't be empty", err.Error()) rule.Table = "$table" _, err = NewFileRouter(rules) - c.Assert(err, IsNil) + require.NoError(t, err) rule.Path = "/tmp/1.sql" _, err = NewFileRouter(rules) - c.Assert(err, ErrorMatches, "can't set both `path` and `pattern` field in \\[\\[mydumper.files\\]\\]") + require.Regexp(t, "can't set both `path` and `pattern` field in \\[\\[mydumper.files\\]\\]", err.Error()) } -func (t *testFileRouterSuite) TestSingleRouteRule(c *C) { +func TestSingleRouteRule(t *testing.T) { rules := []*config.FileRouteRule{ {Pattern: `^(?:[^/]*/)*([^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$1", Table: "$table", Type: "$type", Key: "$key", Compression: "$cp"}, } r, err := NewFileRouter(rules) - c.Assert(err, IsNil) + require.NoError(t, err) inputOutputMap := map[string][]string{ "my_schema.my_table.sql": {"my_schema", "my_table", "", "", "sql"}, @@ -86,13 +84,13 @@ func (t *testFileRouterSuite) TestSingleRouteRule(c *C) { } for path, fields := range inputOutputMap { res, err := r.Route(path) - c.Assert(err, IsNil) + assert.NoError(t, err) compress, e := parseCompressionType(fields[3]) - c.Assert(e, IsNil) + assert.NoError(t, e) ty, e := parseSourceType(fields[4]) - c.Assert(e, IsNil) + assert.NoError(t, e) exp := &RouteResult{filter.Table{Schema: fields[0], Name: fields[1]}, fields[2], compress, ty} - c.Assert(res, DeepEquals, exp) + assert.Equal(t, exp, res) } notMatchPaths := []string{ @@ -104,14 +102,14 @@ func (t *testFileRouterSuite) TestSingleRouteRule(c *C) { } for _, p := range notMatchPaths { res, err := r.Route(p) - c.Assert(res, IsNil) - c.Assert(err, IsNil) + assert.Nil(t, res) + assert.NoError(t, err) } rule := &config.FileRouteRule{Pattern: `^(?:[^/]*/)*([^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?P\w+)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$1", Table: "$table", Type: "$type", Key: "$key", Compression: "$cp"} r, err = NewFileRouter([]*config.FileRouteRule{rule}) - c.Assert(err, IsNil) - c.Assert(r, NotNil) + require.NoError(t, err) + require.NotNil(t, r) invalidMatchPaths := []string{ "my_schema.my_table.sql.gz", "my_schema.my_table.sql.rar", @@ -119,12 +117,12 @@ func (t *testFileRouterSuite) TestSingleRouteRule(c *C) { } for _, p := range invalidMatchPaths { res, err := r.Route(p) - c.Assert(res, IsNil) - c.Assert(err, NotNil) + assert.Nil(t, res) + assert.Error(t, err) } } -func (t *testFileRouterSuite) TestMultiRouteRule(c *C) { +func TestMultiRouteRule(t *testing.T) { // multi rule don't intersect with each other rules := []*config.FileRouteRule{ {Pattern: `(?:[^/]*/)*([^/.]+)-schema-create\.sql`, Schema: "$1", Type: SchemaSchema}, @@ -134,7 +132,7 @@ func (t *testFileRouterSuite) TestMultiRouteRule(c *C) { } r, err := NewFileRouter(rules) - c.Assert(err, IsNil) + require.NoError(t, err) inputOutputMap := map[string][]string{ "test-schema-create.sql": {"test", "", "", "", SchemaSchema}, @@ -148,16 +146,16 @@ func (t *testFileRouterSuite) TestMultiRouteRule(c *C) { } for path, fields := range inputOutputMap { res, err := r.Route(path) - c.Assert(err, IsNil) + assert.NoError(t, err) if len(fields) == 0 { - c.Assert(res, IsNil) + assert.Nil(t, res) } else { compress, e := parseCompressionType(fields[3]) - c.Assert(e, IsNil) + assert.NoError(t, e) ty, e := parseSourceType(fields[4]) - c.Assert(e, IsNil) + assert.NoError(t, e) exp := &RouteResult{filter.Table{Schema: fields[0], Name: fields[1]}, fields[2], compress, ty} - c.Assert(res, DeepEquals, exp) + assert.Equal(t, exp, res) } } @@ -166,24 +164,24 @@ func (t *testFileRouterSuite) TestMultiRouteRule(c *C) { p := &config.FileRouteRule{Pattern: `^(?P[^/.]+)\.(?P
[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "test_schema", Table: "test_table", Type: "$type", Key: "$key", Compression: "$cp"} rules = append(rules, p) r, err = NewFileRouter(rules) - c.Assert(err, IsNil) + require.NoError(t, err) for path, fields := range inputOutputMap { res, err := r.Route(path) - c.Assert(err, IsNil) + assert.NoError(t, err) if len(fields) == 0 { - c.Assert(res, IsNil) + assert.Nil(t, res) } else { compress, e := parseCompressionType(fields[3]) - c.Assert(e, IsNil) + assert.NoError(t, e) ty, e := parseSourceType(fields[4]) - c.Assert(e, IsNil) + assert.NoError(t, e) exp := &RouteResult{filter.Table{Schema: fields[0], Name: fields[1]}, fields[2], compress, ty} - c.Assert(res, DeepEquals, exp) + assert.Equal(t, exp, res) } } } -func (t *testFileRouterSuite) TestRouteExpanding(c *C) { +func TestRouteExpanding(t *testing.T) { rule := &config.FileRouteRule{ Pattern: `^(?:[^/]*/)*(?P[^/.]+)\.(?P[^./]+)(?:\.(?P[0-9]+))?\.(?Pcsv|sql)(?:\.(?P[A-Za-z0-9]+))?$`, Schema: "$schema", @@ -212,22 +210,22 @@ func (t *testFileRouterSuite) TestRouteExpanding(c *C) { for pat, value := range tablePatternResMap { rule.Table = pat router, err := NewFileRouter([]*config.FileRouteRule{rule}) - c.Assert(err, IsNil) + assert.NoError(t, err) res, err := router.Route(path) - c.Assert(err, IsNil) - c.Assert(res, NotNil) - c.Assert(res.Name, Equals, value) + assert.NoError(t, err) + assert.NotNil(t, res) + assert.Equal(t, value, res.Name) } invalidPatterns := []string{"$1_$schema", "$schema_$table_name", "$6"} for _, pat := range invalidPatterns { rule.Table = pat _, err := NewFileRouter([]*config.FileRouteRule{rule}) - c.Assert(err, NotNil) + assert.Error(t, err) } } -func (t *testFileRouterSuite) TestRouteWithPath(c *C) { +func TestRouteWithPath(t *testing.T) { fileName := "myschema.(my_table$1).000.sql" rule := &config.FileRouteRule{ Path: fileName, @@ -238,18 +236,18 @@ func (t *testFileRouterSuite) TestRouteWithPath(c *C) { } r := *rule router, err := NewFileRouter([]*config.FileRouteRule{&r}) - c.Assert(err, IsNil) + require.NoError(t, err) res, err := router.Route(fileName) - c.Assert(err, IsNil) - c.Assert(res, NotNil) - c.Assert(res.Schema, Equals, rule.Schema) - c.Assert(res.Name, Equals, rule.Table) + require.NoError(t, err) + require.NotNil(t, res) + require.Equal(t, rule.Schema, res.Schema) + require.Equal(t, rule.Table, res.Name) ty, _ := parseSourceType(rule.Type) - c.Assert(res.Type, Equals, ty) - c.Assert(res.Key, Equals, rule.Key) + require.Equal(t, ty, res.Type) + require.Equal(t, rule.Key, res.Key) // replace all '.' by '-', if with plain regex pattern, will still match res, err = router.Route(strings.ReplaceAll(fileName, ".", "-")) - c.Assert(err, IsNil) - c.Assert(res, IsNil) + require.NoError(t, err) + require.Nil(t, res) } diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index 3cf12d6f39d1b..ffb71f916e7e2 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -48,11 +48,10 @@ import ( "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" - "github.com/tikv/pd/server/api" - pdconfig "github.com/tikv/pd/server/config" ) const ( @@ -78,7 +77,7 @@ func (rc *Controller) isSourceInLocal() bool { } func (rc *Controller) getReplicaCount(ctx context.Context) (uint64, error) { - result := &pdconfig.ReplicationConfig{} + result := &pdtypes.ReplicationConfig{} err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdReplicate, &result) if err != nil { return 0, errors.Trace(err) @@ -87,7 +86,7 @@ func (rc *Controller) getReplicaCount(ctx context.Context) (uint64, error) { } func (rc *Controller) getClusterAvail(ctx context.Context) (uint64, error) { - result := &api.StoresInfo{} + result := &pdtypes.StoresInfo{} if err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, result); err != nil { return 0, errors.Trace(err) } @@ -183,7 +182,7 @@ func (rc *Controller) ClusterIsAvailable(ctx context.Context) error { return nil } -func isTiFlash(store *api.MetaStore) bool { +func isTiFlash(store *pdtypes.MetaStore) bool { for _, label := range store.Labels { if label.Key == "engine" && label.Value == "tiflash" { return true @@ -198,7 +197,7 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error { defer func() { rc.checkTemplate.Collect(Critical, passed, message) }() - storeInfo := &api.StoresInfo{} + storeInfo := &pdtypes.StoresInfo{} err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, storeInfo) if err != nil { return errors.Trace(err) @@ -207,19 +206,19 @@ func (rc *Controller) checkEmptyRegion(ctx context.Context) error { return nil } - var result api.RegionsInfo + var result pdtypes.RegionsInfo if err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdEmptyRegions, &result); err != nil { return errors.Trace(err) } regions := make(map[uint64]int) - stores := make(map[uint64]*api.StoreInfo) + stores := make(map[uint64]*pdtypes.StoreInfo) for _, region := range result.Regions { for _, peer := range region.Peers { regions[peer.StoreId]++ } } for _, store := range storeInfo.Stores { - stores[store.Store.StoreID] = store + stores[store.Store.GetId()] = store } tableCount := 0 for _, db := range rc.dbMetas { @@ -275,12 +274,12 @@ func (rc *Controller) checkRegionDistribution(ctx context.Context) error { rc.checkTemplate.Collect(Critical, passed, message) }() - result := &api.StoresInfo{} + result := &pdtypes.StoresInfo{} err := rc.tls.WithHost(rc.cfg.TiDB.PdAddr).GetJSON(ctx, pdStores, result) if err != nil { return errors.Trace(err) } - stores := make([]*api.StoreInfo, 0, len(result.Stores)) + stores := make([]*pdtypes.StoreInfo, 0, len(result.Stores)) for _, store := range result.Stores { if metapb.StoreState(metapb.StoreState_value[store.Store.StateName]) != metapb.StoreState_Up { continue @@ -315,11 +314,11 @@ func (rc *Controller) checkRegionDistribution(ctx context.Context) error { passed = false message = fmt.Sprintf("Region distribution is unbalanced, the ratio of the regions count of the store(%v) "+ "with least regions(%v) to the store(%v) with most regions(%v) is %v, but we expect it must not be less than %v", - minStore.Store.StoreID, minStore.Status.RegionCount, maxStore.Store.StoreID, maxStore.Status.RegionCount, ratio, errorRegionCntMinMaxRatio) + minStore.Store.GetId(), minStore.Status.RegionCount, maxStore.Store.GetId(), maxStore.Status.RegionCount, ratio, errorRegionCntMinMaxRatio) } else if ratio < warnRegionCntMinMaxRatio { message = fmt.Sprintf("Region distribution is unbalanced, the ratio of the regions count of the store(%v) "+ "with least regions(%v) to the store(%v) with most regions(%v) is %v, but we expect it should not be less than %v", - minStore.Store.StoreID, minStore.Status.RegionCount, maxStore.Store.StoreID, maxStore.Status.RegionCount, ratio, warnRegionCntMinMaxRatio) + minStore.Store.GetId(), minStore.Status.RegionCount, maxStore.Store.GetId(), maxStore.Status.RegionCount, ratio, warnRegionCntMinMaxRatio) } return nil } diff --git a/br/pkg/lightning/restore/check_info_test.go b/br/pkg/lightning/restore/check_info_test.go index c679298f6a612..4479cbbdc3a25 100644 --- a/br/pkg/lightning/restore/check_info_test.go +++ b/br/pkg/lightning/restore/check_info_test.go @@ -20,10 +20,11 @@ import ( "fmt" "os" "path/filepath" + "testing" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" + "github.com/stretchr/testify/require" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/config" @@ -39,17 +40,14 @@ import ( tmock "github.com/pingcap/tidb/util/mock" ) -var _ = Suite(&checkInfoSuite{}) - -type checkInfoSuite struct{} - const passed CheckType = "pass" -func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { - dir := c.MkDir() +func TestCheckCSVHeader(t *testing.T) { + dir := t.TempDir() + ctx := context.Background() mockStore, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) type tableSource struct { Name string @@ -363,9 +361,9 @@ func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { for _, tbl := range tbls { node, err := p.ParseOneStmt(tbl.SQL, "", "") - c.Assert(err, IsNil) + require.NoError(t, err) core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) - c.Assert(err, IsNil) + require.NoError(t, err) core.State = model.StatePublic dbInfo.Tables[tbl.Name] = &checkpoints.TidbTableInfo{ ID: core.ID, @@ -378,7 +376,7 @@ func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { for i, s := range tbl.Sources { fileName := fmt.Sprintf("%s.%s.%d.csv", db, tbl.Name, i) err = os.WriteFile(filepath.Join(dir, fileName), []byte(s), 0o644) - c.Assert(err, IsNil) + require.NoError(t, err) fileInfos = append(fileInfos, mydump.FileInfo{ FileMeta: mydump.SourceFileMeta{ Path: fileName, @@ -400,15 +398,16 @@ func (s *checkInfoSuite) TestCheckCSVHeader(c *C) { } err := rc.checkCSVHeader(ctx, dbMetas) - c.Assert(err, IsNil) + require.NoError(t, err) if ca.level != passed { - c.Assert(rc.checkTemplate.FailedCount(ca.level), Equals, 1) + require.Equal(t, 1, rc.checkTemplate.FailedCount(ca.level)) } } } -func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { - dir := c.MkDir() +func TestCheckTableEmpty(t *testing.T) { + dir := t.TempDir() + cfg := config.NewConfig() cfg.Checkpoint.Enable = false dbMetas := []*mydump.MDDatabaseMeta{ @@ -447,17 +446,17 @@ func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { // test tidb will do nothing rc.cfg.TikvImporter.Backend = config.BackendTiDB err := rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) // test incremental mode rc.cfg.TikvImporter.Backend = config.BackendLocal rc.cfg.TikvImporter.IncrementalImport = true err = rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) rc.cfg.TikvImporter.IncrementalImport = false db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) mock.MatchExpectationsInOrder(false) rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). @@ -468,12 +467,12 @@ func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) // not error, need not to init check template err = rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) // single table contains data db, mock, err = sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) mock.MatchExpectationsInOrder(false) mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). @@ -484,15 +483,16 @@ func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1)) rc.checkTemplate = NewSimpleTemplate() err = rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) + tmpl := rc.checkTemplate.(*SimpleTemplate) - c.Assert(len(tmpl.criticalMsgs), Equals, 1) - c.Assert(tmpl.criticalMsgs[0], Matches, "table\\(s\\) \\[`test2`.`tbl1`\\] are not empty") + require.Equal(t, 1, len(tmpl.criticalMsgs)) + require.Equal(t, "table(s) [`test2`.`tbl1`] are not empty", tmpl.criticalMsgs[0]) // multi tables contains data db, mock, err = sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) mock.MatchExpectationsInOrder(false) mock.ExpectQuery("select 1 from `test1`.`tbl1` limit 1"). @@ -503,11 +503,12 @@ func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { WillReturnRows(sqlmock.NewRows([]string{""}).AddRow(1)) rc.checkTemplate = NewSimpleTemplate() err = rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) + tmpl = rc.checkTemplate.(*SimpleTemplate) - c.Assert(len(tmpl.criticalMsgs), Equals, 1) - c.Assert(tmpl.criticalMsgs[0], Matches, "table\\(s\\) \\[`test1`.`tbl1`, `test2`.`tbl1`\\] are not empty") + require.Equal(t, 1, len(tmpl.criticalMsgs)) + require.Equal(t, "table(s) [`test1`.`tbl1`, `test2`.`tbl1`] are not empty", tmpl.criticalMsgs[0]) // init checkpoint with only two of the three tables dbInfos := map[string]*checkpoints.TidbDBInfo{ @@ -531,25 +532,26 @@ func (s *checkInfoSuite) TestCheckTableEmpty(c *C) { rc.cfg.Checkpoint.Enable = true rc.checkpointsDB = checkpoints.NewFileCheckpointsDB(filepath.Join(dir, "cp.pb")) err = rc.checkpointsDB.Initialize(ctx, cfg, dbInfos) - c.Check(err, IsNil) + require.NoError(t, err) db, mock, err = sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) rc.tidbGlue = glue.NewExternalTiDBGlue(db, mysql.ModeNone) // only need to check the one that is not in checkpoint mock.ExpectQuery("select 1 from `test1`.`tbl2` limit 1"). WillReturnRows(sqlmock.NewRows([]string{""}).RowError(0, sql.ErrNoRows)) err = rc.checkTableEmpty(ctx) - c.Assert(err, IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) } -func (s *checkInfoSuite) TestLocalResource(c *C) { - dir := c.MkDir() +func TestLocalResource(t *testing.T) { + dir := t.TempDir() + mockStore, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) + require.NoError(t, err) err = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/common/GetStorageSize", "return(2048)") - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/common/GetStorageSize") }() @@ -567,28 +569,28 @@ func (s *checkInfoSuite) TestLocalResource(c *C) { // 1. source-size is smaller than disk-size, won't trigger error information rc.checkTemplate = NewSimpleTemplate() err = rc.localResource(1000) - c.Assert(err, IsNil) + require.NoError(t, err) tmpl := rc.checkTemplate.(*SimpleTemplate) - c.Assert(tmpl.warnFailedCount, Equals, 1) - c.Assert(tmpl.criticalFailedCount, Equals, 0) - c.Assert(tmpl.normalMsgs[1], Matches, "local disk resources are rich, estimate sorted data size 1000B, local available is 2KiB") + require.Equal(t, 1, tmpl.warnFailedCount) + require.Equal(t, 0, tmpl.criticalFailedCount) + require.Equal(t, "local disk resources are rich, estimate sorted data size 1000B, local available is 2KiB", tmpl.normalMsgs[1]) // 2. source-size is bigger than disk-size, with default disk-quota will trigger a critical error rc.checkTemplate = NewSimpleTemplate() err = rc.localResource(4096) - c.Assert(err, IsNil) + require.NoError(t, err) tmpl = rc.checkTemplate.(*SimpleTemplate) - c.Assert(tmpl.warnFailedCount, Equals, 1) - c.Assert(tmpl.criticalFailedCount, Equals, 1) - c.Assert(tmpl.criticalMsgs[0], Matches, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB, please set `tikv-importer.disk-quota` to a smaller value than 2KiB or change `mydumper.sorted-kv-dir` to another disk with enough space to finish imports") + require.Equal(t, 1, tmpl.warnFailedCount) + require.Equal(t, 1, tmpl.criticalFailedCount) + require.Equal(t, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB, please set `tikv-importer.disk-quota` to a smaller value than 2KiB or change `mydumper.sorted-kv-dir` to another disk with enough space to finish imports", tmpl.criticalMsgs[0]) // 3. source-size is bigger than disk-size, with a vaild disk-quota will trigger a warning rc.checkTemplate = NewSimpleTemplate() rc.cfg.TikvImporter.DiskQuota = config.ByteSize(1024) err = rc.localResource(4096) - c.Assert(err, IsNil) + require.NoError(t, err) tmpl = rc.checkTemplate.(*SimpleTemplate) - c.Assert(tmpl.warnFailedCount, Equals, 1) - c.Assert(tmpl.criticalFailedCount, Equals, 0) - c.Assert(tmpl.normalMsgs[1], Matches, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB,we will use disk-quota \\(size: 1KiB\\) to finish imports, which may slow down import") + require.Equal(t, 1, tmpl.warnFailedCount) + require.Equal(t, 0, tmpl.criticalFailedCount) + require.Equal(t, "local disk space may not enough to finish import, estimate sorted data size is 4KiB, but local available is 2KiB,we will use disk-quota (size: 1KiB) to finish imports, which may slow down import", tmpl.normalMsgs[1]) } diff --git a/br/pkg/lightning/restore/checksum.go b/br/pkg/lightning/restore/checksum.go index 5fc3489b370e0..85b3c4bf1f1c4 100644 --- a/br/pkg/lightning/restore/checksum.go +++ b/br/pkg/lightning/restore/checksum.go @@ -20,7 +20,6 @@ import ( "database/sql" "fmt" "sync" - "sync/atomic" "time" "github.com/google/uuid" @@ -40,6 +39,7 @@ import ( "github.com/pingcap/tipb/go-tipb" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -373,7 +373,7 @@ type gcTTLManager struct { currentTS uint64 serviceID string // 0 for not start, otherwise started - started uint32 + started atomic.Bool } func newGCTTLManager(pdClient pd.Client) gcTTLManager { @@ -385,7 +385,7 @@ func newGCTTLManager(pdClient pd.Client) gcTTLManager { func (m *gcTTLManager) addOneJob(ctx context.Context, table string, ts uint64) error { // start gc ttl loop if not started yet. - if atomic.CompareAndSwapUint32(&m.started, 0, 1) { + if m.started.CAS(false, true) { m.start(ctx) } m.lock.Lock() diff --git a/br/pkg/lightning/restore/checksum_test.go b/br/pkg/lightning/restore/checksum_test.go index 988a41eecbbfa..dd2419b381e25 100644 --- a/br/pkg/lightning/restore/checksum_test.go +++ b/br/pkg/lightning/restore/checksum_test.go @@ -7,11 +7,10 @@ import ( "sort" "strings" "sync" - "sync/atomic" + "testing" "time" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/errors" . "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/ddl" @@ -21,23 +20,25 @@ import ( "github.com/pingcap/tidb/util" tmock "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" + "go.uber.org/atomic" ) -var _ = Suite(&checksumSuite{}) - -type checksumSuite struct{} - func MockDoChecksumCtx(db *sql.DB) context.Context { ctx := context.Background() manager := newTiDBChecksumExecutor(db) return context.WithValue(ctx, &checksumManagerKey, manager) } -func (s *checksumSuite) TestDoChecksum(c *C) { +func TestDoChecksum(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + defer func() { + require.NoError(t, db.Close()) + require.NoError(t, mock.ExpectationsWereMet()) + }() mock.ExpectQuery("\\QSELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME = 'tikv_gc_life_time'\\E"). WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) @@ -56,22 +57,23 @@ func (s *checksumSuite) TestDoChecksum(c *C) { ctx := MockDoChecksumCtx(db) checksum, err := DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) - c.Assert(err, IsNil) - c.Assert(*checksum, DeepEquals, RemoteChecksum{ + require.NoError(t, err) + require.Equal(t, RemoteChecksum{ Schema: "test", Table: "t", Checksum: 8520875019404689597, TotalKVs: 7296873, TotalBytes: 357601387, - }) - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + }, *checksum) } -func (s *checksumSuite) TestDoChecksumParallel(c *C) { +func TestDoChecksumParallel(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + defer func() { + require.NoError(t, db.Close()) + require.NoError(t, mock.ExpectationsWereMet()) + }() mock.ExpectQuery("\\QSELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME = 'tikv_gc_life_time'\\E"). WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) @@ -99,25 +101,26 @@ func (s *checksumSuite) TestDoChecksumParallel(c *C) { for i := 0; i < 5; i++ { wg.Run(func() { checksum, err := DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) - c.Assert(err, IsNil) - c.Assert(*checksum, DeepEquals, RemoteChecksum{ + require.NoError(t, err) + require.Equal(t, RemoteChecksum{ Schema: "test", Table: "t", Checksum: 8520875019404689597, TotalKVs: 7296873, TotalBytes: 357601387, - }) + }, *checksum) }) } wg.Wait() - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) } -func (s *checksumSuite) TestIncreaseGCLifeTimeFail(c *C) { +func TestIncreaseGCLifeTimeFail(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + defer func() { + require.NoError(t, db.Close()) + require.NoError(t, mock.ExpectationsWereMet()) + }() for i := 0; i < 5; i++ { mock.ExpectQuery("\\QSELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME = 'tikv_gc_life_time'\\E"). @@ -138,19 +141,16 @@ func (s *checksumSuite) TestIncreaseGCLifeTimeFail(c *C) { for i := 0; i < 5; i++ { wg.Run(func() { _, errChecksum := DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) - c.Assert(errChecksum, ErrorMatches, "update GC lifetime failed: update gc error: context canceled") + require.Equal(t, "update GC lifetime failed: update gc error: context canceled", errChecksum.Error()) }) } wg.Wait() _, err = db.Exec("\\QUPDATE mysql.tidb SET VARIABLE_VALUE = ? WHERE VARIABLE_NAME = 'tikv_gc_life_time'\\E", "10m") - c.Assert(err, IsNil) - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.NoError(t, err) } -func (s *checksumSuite) TestDoChecksumWithTikv(c *C) { +func TestDoChecksumWithTikv(t *testing.T) { // set up mock tikv checksum manager pdClient := &testPDClient{} resp := tipb.ChecksumResponse{Checksum: 123, TotalKvs: 10, TotalBytes: 1000} @@ -160,9 +160,9 @@ func (s *checksumSuite) TestDoChecksumWithTikv(c *C) { p := parser.New() se := tmock.NewContext() node, err := p.ParseOneStmt("CREATE TABLE `t1` (`c1` varchar(5) NOT NULL)", "utf8mb4", "utf8mb4_bin") - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 999) - c.Assert(err, IsNil) + require.NoError(t, err) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -176,24 +176,28 @@ func (s *checksumSuite) TestDoChecksumWithTikv(c *C) { _, err = DoChecksum(subCtx, &TidbTableInfo{DB: "test", Name: "t", Core: tableInfo}) // with max error retry < maxErrorRetryCount, the checksum can success if i >= maxErrorRetryCount { - c.Assert(err, ErrorMatches, "tikv timeout") + require.Equal(t, "tikv timeout", err.Error()) continue } else { - c.Assert(err, IsNil) + require.NoError(t, err) } // after checksum, safepint should be small than start ts ts := pdClient.currentSafePoint() // 1ms for the schedule deviation - c.Assert(ts <= startTS+1, IsTrue) - c.Assert(atomic.LoadUint32(&checksumExec.manager.started) > 0, IsTrue) - c.Assert(len(checksumExec.manager.tableGCSafeTS), Equals, 0) + require.True(t, ts <= startTS+1) + require.True(t, checksumExec.manager.started.Load()) + require.Equal(t, 0, len(checksumExec.manager.tableGCSafeTS)) } } -func (s *checksumSuite) TestDoChecksumWithErrorAndLongOriginalLifetime(c *C) { +func TestDoChecksumWithErrorAndLongOriginalLifetime(t *testing.T) { db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + defer func() { + require.NoError(t, db.Close()) + require.NoError(t, mock.ExpectationsWereMet()) + }() mock.ExpectQuery("\\QSELECT VARIABLE_VALUE FROM mysql.tidb WHERE VARIABLE_NAME = 'tikv_gc_life_time'\\E"). WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("300h")) @@ -206,10 +210,7 @@ func (s *checksumSuite) TestDoChecksumWithErrorAndLongOriginalLifetime(c *C) { ctx := MockDoChecksumCtx(db) _, err = DoChecksum(ctx, &TidbTableInfo{DB: "test", Name: "t"}) - c.Assert(err, ErrorMatches, "compute remote checksum failed: mock syntax error.*") - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.Regexp(t, "compute remote checksum failed: mock syntax error.*", err.Error()) } type safePointTTL struct { @@ -221,7 +222,7 @@ type safePointTTL struct { type testPDClient struct { sync.Mutex pd.Client - count int32 + count atomic.Int32 gcSafePoint []safePointTTL } @@ -245,7 +246,7 @@ func (c *testPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID s if !strings.HasPrefix(serviceID, "lightning") { panic("service ID must start with 'lightning'") } - atomic.AddInt32(&c.count, 1) + c.count.Add(1) c.Lock() idx := sort.Search(len(c.gcSafePoint), func(i int) bool { return c.gcSafePoint[i].safePoint >= safePoint @@ -267,10 +268,10 @@ func (c *testPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID s return c.currentSafePoint(), nil } -func (s *checksumSuite) TestGcTTLManagerSingle(c *C) { +func TestGcTTLManagerSingle(t *testing.T) { pdClient := &testPDClient{} manager := newGCTTLManager(pdClient) - c.Assert(manager.serviceID, Not(Equals), "") + require.NotEqual(t, "", manager.serviceID) ctx, cancel := context.WithCancel(context.Background()) defer cancel() oldTTL := serviceSafePointTTL @@ -281,56 +282,56 @@ func (s *checksumSuite) TestGcTTLManagerSingle(c *C) { }() err := manager.addOneJob(ctx, "test", uint64(time.Now().Unix())) - c.Assert(err, IsNil) + require.NoError(t, err) time.Sleep(2*time.Second + 10*time.Millisecond) // after 2 seconds, must at least update 5 times - val := atomic.LoadInt32(&pdClient.count) - c.Assert(val, GreaterEqual, int32(5)) + val := pdClient.count.Load() + require.GreaterOrEqual(t, val, int32(5)) // after remove the job, there are no job remain, gc ttl needn't to be updated manager.removeOneJob("test") time.Sleep(10 * time.Millisecond) - val = atomic.LoadInt32(&pdClient.count) + val = pdClient.count.Load() time.Sleep(1*time.Second + 10*time.Millisecond) - c.Assert(atomic.LoadInt32(&pdClient.count), Equals, val) + require.Equal(t, val, pdClient.count.Load()) } -func (s *checksumSuite) TestGcTTLManagerMulti(c *C) { +func TestGcTTLManagerMulti(t *testing.T) { manager := newGCTTLManager(&testPDClient{}) ctx := context.Background() for i := uint64(1); i <= 5; i++ { err := manager.addOneJob(ctx, fmt.Sprintf("test%d", i), i) - c.Assert(err, IsNil) - c.Assert(manager.currentTS, Equals, uint64(1)) + require.NoError(t, err) + require.Equal(t, uint64(1), manager.currentTS) } manager.removeOneJob("test2") - c.Assert(manager.currentTS, Equals, uint64(1)) + require.Equal(t, uint64(1), manager.currentTS) manager.removeOneJob("test1") - c.Assert(manager.currentTS, Equals, uint64(3)) + require.Equal(t, uint64(3), manager.currentTS) manager.removeOneJob("test3") - c.Assert(manager.currentTS, Equals, uint64(4)) + require.Equal(t, uint64(4), manager.currentTS) manager.removeOneJob("test4") - c.Assert(manager.currentTS, Equals, uint64(5)) + require.Equal(t, uint64(5), manager.currentTS) manager.removeOneJob("test5") - c.Assert(manager.currentTS, Equals, uint64(0)) + require.Equal(t, uint64(0), manager.currentTS) } -func (s *checksumSuite) TestPdServiceID(c *C) { +func TestPdServiceID(t *testing.T) { pdCli := &testPDClient{} gcTTLManager1 := newGCTTLManager(pdCli) - c.Assert(gcTTLManager1.serviceID, Matches, "lightning-.*") + require.Regexp(t, "lightning-.*", gcTTLManager1.serviceID) gcTTLManager2 := newGCTTLManager(pdCli) - c.Assert(gcTTLManager2.serviceID, Matches, "lightning-.*") + require.Regexp(t, "lightning-.*", gcTTLManager2.serviceID) - c.Assert(gcTTLManager1.serviceID != gcTTLManager2.serviceID, IsTrue) + require.True(t, gcTTLManager1.serviceID != gcTTLManager2.serviceID) } type mockResponse struct { diff --git a/br/pkg/lightning/restore/chunk_restore_test.go b/br/pkg/lightning/restore/chunk_restore_test.go new file mode 100644 index 0000000000000..4b238971fdbd5 --- /dev/null +++ b/br/pkg/lightning/restore/chunk_restore_test.go @@ -0,0 +1,568 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package restore + +import ( + "context" + "os" + "path/filepath" + "testing" + + "github.com/golang/mock/gomock" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/import_kvpb" + "github.com/pingcap/tidb/br/pkg/lightning/backend" + "github.com/pingcap/tidb/br/pkg/lightning/backend/importer" + "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/backend/tidb" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/errormanager" + "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/lightning/worker" + "github.com/pingcap/tidb/br/pkg/mock" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type chunkRestoreSuite struct { + suite.Suite + tableRestoreSuiteBase + cr *chunkRestore +} + +func TestChunkRestoreSuite(t *testing.T) { + suite.Run(t, new(chunkRestoreSuite)) +} + +func (s *chunkRestoreSuite) SetupSuite() { + s.setupSuite(s.T()) +} + +func (s *chunkRestoreSuite) SetupTest() { + s.setupTest(s.T()) + + ctx := context.Background() + w := worker.NewPool(ctx, 5, "io") + + chunk := checkpoints.ChunkCheckpoint{ + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[1].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 18, + RowIDMax: 36, + }, + } + + var err error + s.cr, err = newChunkRestore(context.Background(), 1, s.cfg, &chunk, w, s.store, nil) + require.NoError(s.T(), err) +} + +func (s *chunkRestoreSuite) TearDownTest() { + s.cr.close() +} + +func (s *chunkRestoreSuite) TestDeliverLoopCancel() { + rc := &Controller{backend: importer.NewMockImporter(nil, "")} + + ctx, cancel := context.WithCancel(context.Background()) + kvsCh := make(chan []deliveredKVs) + go cancel() + _, err := s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, nil, nil, rc) + require.Equal(s.T(), context.Canceled, errors.Cause(err)) +} + +func (s *chunkRestoreSuite) TestDeliverLoopEmptyData() { + ctx := context.Background() + + // Open two mock engines. + + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockBackend := mock.NewMockBackend(controller) + importer := backend.MakeBackend(mockBackend) + + mockBackend.EXPECT().OpenEngine(ctx, gomock.Any(), gomock.Any()).Return(nil).Times(2) + mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).AnyTimes() + mockWriter := mock.NewMockEngineWriter(controller) + mockBackend.EXPECT().LocalWriter(ctx, gomock.Any(), gomock.Any()).Return(mockWriter, nil).AnyTimes() + mockWriter.EXPECT(). + AppendRows(ctx, gomock.Any(), gomock.Any(), gomock.Any()). + Return(nil).AnyTimes() + + dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) + require.NoError(s.T(), err) + dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) + require.NoError(s.T(), err) + indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + + // Deliver nothing. + + cfg := &config.Config{} + rc := &Controller{cfg: cfg, backend: importer, diskQuotaLock: newDiskQuotaLock()} + + kvsCh := make(chan []deliveredKVs, 1) + kvsCh <- []deliveredKVs{} + _, err = s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, dataWriter, indexWriter, rc) + require.NoError(s.T(), err) +} + +func (s *chunkRestoreSuite) TestDeliverLoop() { + ctx := context.Background() + kvsCh := make(chan []deliveredKVs) + mockCols := []string{"c1", "c2"} + + // Open two mock engines. + + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockBackend := mock.NewMockBackend(controller) + importer := backend.MakeBackend(mockBackend) + + mockBackend.EXPECT().OpenEngine(ctx, gomock.Any(), gomock.Any()).Return(nil).Times(2) + // avoid return the same object at each call + mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).Times(1) + mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).Times(1) + mockWriter := mock.NewMockEngineWriter(controller) + mockBackend.EXPECT().LocalWriter(ctx, gomock.Any(), gomock.Any()).Return(mockWriter, nil).AnyTimes() + mockWriter.EXPECT().IsSynced().Return(true).AnyTimes() + + dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) + require.NoError(s.T(), err) + indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) + require.NoError(s.T(), err) + + dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + + // Set up the expected API calls to the data engine... + + mockWriter.EXPECT(). + AppendRows(ctx, s.tr.tableName, mockCols, kv.MakeRowsFromKvPairs([]common.KvPair{ + { + Key: []byte("txxxxxxxx_ryyyyyyyy"), + Val: []byte("value1"), + }, + { + Key: []byte("txxxxxxxx_rwwwwwwww"), + Val: []byte("value2"), + }, + })). + Return(nil) + + // ... and the index engine. + // + // Note: This test assumes data engine is written before the index engine. + + mockWriter.EXPECT(). + AppendRows(ctx, s.tr.tableName, mockCols, kv.MakeRowsFromKvPairs([]common.KvPair{ + { + Key: []byte("txxxxxxxx_izzzzzzzz"), + Val: []byte("index1"), + }, + })). + Return(nil) + + // Now actually start the delivery loop. + + saveCpCh := make(chan saveCp, 2) + go func() { + kvsCh <- []deliveredKVs{ + { + kvs: kv.MakeRowFromKvPairs([]common.KvPair{ + { + Key: []byte("txxxxxxxx_ryyyyyyyy"), + Val: []byte("value1"), + }, + { + Key: []byte("txxxxxxxx_rwwwwwwww"), + Val: []byte("value2"), + }, + { + Key: []byte("txxxxxxxx_izzzzzzzz"), + Val: []byte("index1"), + }, + }), + columns: mockCols, + offset: 12, + rowID: 76, + }, + } + kvsCh <- []deliveredKVs{} + close(kvsCh) + }() + + cfg := &config.Config{} + rc := &Controller{cfg: cfg, saveCpCh: saveCpCh, backend: importer, diskQuotaLock: newDiskQuotaLock()} + + _, err = s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, dataWriter, indexWriter, rc) + require.NoError(s.T(), err) + require.Len(s.T(), saveCpCh, 2) + require.Equal(s.T(), int64(12), s.cr.chunk.Chunk.Offset) + require.Equal(s.T(), int64(76), s.cr.chunk.Chunk.PrevRowIDMax) + require.Equal(s.T(), uint64(3), s.cr.chunk.Checksum.SumKVS()) +} + +func (s *chunkRestoreSuite) TestEncodeLoop() { + ctx := context.Background() + kvsCh := make(chan []deliveredKVs, 2) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567895, + }) + require.NoError(s.T(), err) + cfg := config.NewConfig() + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.NoError(s.T(), err) + require.Len(s.T(), kvsCh, 2) + + kvs := <-kvsCh + require.Len(s.T(), kvs, 1) + require.Equal(s.T(), int64(19), kvs[0].rowID) + require.Equal(s.T(), int64(36), kvs[0].offset) + require.Equal(s.T(), []string(nil), kvs[0].columns) + + kvs = <-kvsCh + require.Equal(s.T(), 0, len(kvs)) +} + +func (s *chunkRestoreSuite) TestEncodeLoopCanceled() { + ctx, cancel := context.WithCancel(context.Background()) + kvsCh := make(chan []deliveredKVs) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567896, + }) + require.NoError(s.T(), err) + + go cancel() + cfg := config.NewConfig() + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.Equal(s.T(), context.Canceled, errors.Cause(err)) + require.Len(s.T(), kvsCh, 0) +} + +func (s *chunkRestoreSuite) TestEncodeLoopForcedError() { + ctx := context.Background() + kvsCh := make(chan []deliveredKVs, 2) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567897, + }) + require.NoError(s.T(), err) + + // close the chunk so reading it will result in the "file already closed" error. + s.cr.parser.Close() + + cfg := config.NewConfig() + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.Regexp(s.T(), `in file .*[/\\]?db\.table\.2\.sql:0 at offset 0:.*file already closed`, err.Error()) + require.Len(s.T(), kvsCh, 0) +} + +func (s *chunkRestoreSuite) TestEncodeLoopDeliverLimit() { + ctx := context.Background() + kvsCh := make(chan []deliveredKVs, 4) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567898, + }) + require.NoError(s.T(), err) + + dir := s.T().TempDir() + + fileName := "db.limit.000.csv" + err = os.WriteFile(filepath.Join(dir, fileName), []byte("1,2,3\r\n4,5,6\r\n7,8,9\r"), 0o644) + require.NoError(s.T(), err) + + store, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + cfg := config.NewConfig() + + reader, err := store.Open(ctx, fileName) + require.NoError(s.T(), err) + w := worker.NewPool(ctx, 1, "io") + p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, false, nil) + require.NoError(s.T(), err) + s.cr.parser = p + + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + require.NoError(s.T(), failpoint.Enable( + "github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size", "return(110000000)")) + defer failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size") + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.NoError(s.T(), err) + + // we have 3 kvs total. after the failpoint injected. + // we will send one kv each time. + count := 0 + for { + kvs, ok := <-kvsCh + if !ok { + break + } + count += 1 + if count <= 3 { + require.Len(s.T(), kvs, 1) + } + if count == 4 { + // we will send empty kvs before encodeLoop exists + // so, we can receive 4 batch and 1 is empty + require.Len(s.T(), kvs, 0) + break + } + } +} + +func (s *chunkRestoreSuite) TestEncodeLoopDeliverErrored() { + ctx := context.Background() + kvsCh := make(chan []deliveredKVs) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567898, + }) + require.NoError(s.T(), err) + + go func() { + deliverCompleteCh <- deliverResult{ + err: errors.New("fake deliver error"), + } + }() + cfg := config.NewConfig() + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.Equal(s.T(), "fake deliver error", err.Error()) + require.Len(s.T(), kvsCh, 0) +} + +func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch() { + dir := s.T().TempDir() + + fileName := "db.table.000.csv" + err := os.WriteFile(filepath.Join(dir, fileName), []byte("1,2\r\n4,5,6,7\r\n"), 0o644) + require.NoError(s.T(), err) + + store, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + + ctx := context.Background() + cfg := config.NewConfig() + errorMgr := errormanager.New(nil, cfg) + rc := &Controller{pauser: DeliverPauser, cfg: cfg, errorMgr: errorMgr} + + reader, err := store.Open(ctx, fileName) + require.NoError(s.T(), err) + w := worker.NewPool(ctx, 5, "io") + p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, false, nil) + require.NoError(s.T(), err) + + err = s.cr.parser.Close() + require.NoError(s.T(), err) + s.cr.parser = p + + kvsCh := make(chan []deliveredKVs, 2) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errorMgr).NewEncoder( + s.tr.encTable, + &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567895, + }) + require.NoError(s.T(), err) + defer kvEncoder.Close() + + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.Equal(s.T(), "in file db.table.2.sql:0 at offset 4: column count mismatch, expected 3, got 2", err.Error()) + require.Len(s.T(), kvsCh, 0) +} + +func (s *chunkRestoreSuite) TestEncodeLoopIgnoreColumnsCSV() { + cases := []struct { + s string + ignoreColumns []*config.IgnoreColumns + kvs deliveredKVs + header bool + }{ + { + "1,2,3\r\n4,5,6\r\n", + []*config.IgnoreColumns{ + { + DB: "db", + Table: "table", + Columns: []string{"a"}, + }, + }, + deliveredKVs{ + rowID: 1, + offset: 6, + columns: []string{"b", "c"}, + }, + false, + }, + { + "b,c\r\n2,3\r\n5,6\r\n", + []*config.IgnoreColumns{ + { + TableFilter: []string{"db*.tab*"}, + Columns: []string{"b"}, + }, + }, + deliveredKVs{ + rowID: 1, + offset: 9, + columns: []string{"c"}, + }, + true, + }, + } + + for _, cs := range cases { + // reset test + s.SetupTest() + s.testEncodeLoopIgnoreColumnsCSV(cs.s, cs.ignoreColumns, cs.kvs, cs.header) + } +} + +func (s *chunkRestoreSuite) testEncodeLoopIgnoreColumnsCSV( + f string, + ignoreColumns []*config.IgnoreColumns, + deliverKV deliveredKVs, + header bool, +) { + dir := s.T().TempDir() + + fileName := "db.table.000.csv" + err := os.WriteFile(filepath.Join(dir, fileName), []byte(f), 0o644) + require.NoError(s.T(), err) + + store, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + + ctx := context.Background() + cfg := config.NewConfig() + cfg.Mydumper.IgnoreColumns = ignoreColumns + cfg.Mydumper.CSV.Header = header + rc := &Controller{pauser: DeliverPauser, cfg: cfg} + + reader, err := store.Open(ctx, fileName) + require.NoError(s.T(), err) + w := worker.NewPool(ctx, 5, "io") + p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, cfg.Mydumper.CSV.Header, nil) + require.NoError(s.T(), err) + + err = s.cr.parser.Close() + require.NoError(s.T(), err) + s.cr.parser = p + + kvsCh := make(chan []deliveredKVs, 2) + deliverCompleteCh := make(chan deliverResult) + kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())).NewEncoder( + s.tr.encTable, + &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 1234567895, + }) + require.NoError(s.T(), err) + defer kvEncoder.Close() + + _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) + require.NoError(s.T(), err) + require.Len(s.T(), kvsCh, 2) + + kvs := <-kvsCh + require.Len(s.T(), kvs, 2) + require.Equal(s.T(), deliverKV.rowID, kvs[0].rowID) + require.Equal(s.T(), deliverKV.offset, kvs[0].offset) + require.Equal(s.T(), deliverKV.columns, kvs[0].columns) + + kvs = <-kvsCh + require.Equal(s.T(), 0, len(kvs)) +} + +func (s *chunkRestoreSuite) TestRestore() { + ctx := context.Background() + + // Open two mock engines + + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockClient := mock.NewMockImportKVClient(controller) + mockDataWriter := mock.NewMockImportKV_WriteEngineClient(controller) + mockIndexWriter := mock.NewMockImportKV_WriteEngineClient(controller) + importer := importer.NewMockImporter(mockClient, "127.0.0.1:2379") + + mockClient.EXPECT().OpenEngine(ctx, gomock.Any()).Return(nil, nil) + mockClient.EXPECT().OpenEngine(ctx, gomock.Any()).Return(nil, nil) + + dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) + require.NoError(s.T(), err) + indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) + require.NoError(s.T(), err) + dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) + require.NoError(s.T(), err) + + // Expected API sequence + // (we don't care about the actual content, this would be checked in the integrated tests) + + mockClient.EXPECT().WriteEngine(ctx).Return(mockDataWriter, nil) + mockDataWriter.EXPECT().Send(gomock.Any()).Return(nil) + mockDataWriter.EXPECT().Send(gomock.Any()).DoAndReturn(func(req *import_kvpb.WriteEngineRequest) error { + require.Len(s.T(), req.GetBatch().GetMutations(), 1) + return nil + }) + mockDataWriter.EXPECT().CloseAndRecv().Return(nil, nil) + + mockClient.EXPECT().WriteEngine(ctx).Return(mockIndexWriter, nil) + mockIndexWriter.EXPECT().Send(gomock.Any()).Return(nil) + mockIndexWriter.EXPECT().Send(gomock.Any()).DoAndReturn(func(req *import_kvpb.WriteEngineRequest) error { + require.Len(s.T(), req.GetBatch().GetMutations(), 1) + return nil + }) + mockIndexWriter.EXPECT().CloseAndRecv().Return(nil, nil) + + // Now actually start the restore loop. + + saveCpCh := make(chan saveCp, 2) + err = s.cr.restore(ctx, s.tr, 0, dataWriter, indexWriter, &Controller{ + cfg: s.cfg, + saveCpCh: saveCpCh, + backend: importer, + pauser: DeliverPauser, + diskQuotaLock: newDiskQuotaLock(), + }) + require.NoError(s.T(), err) + require.Len(s.T(), saveCpCh, 2) +} diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 58d8c59966a6b..b0b08c905a1e5 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -180,8 +180,11 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64 } needAutoID := common.TableHasAutoRowID(m.tr.tableInfo.Core) || m.tr.tableInfo.Core.GetAutoIncrementColInfo() != nil || m.tr.tableInfo.Core.ContainsAutoRandomBits() err = exec.Transact(ctx, "init table allocator base", func(ctx context.Context, tx *sql.Tx) error { - query := fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName) // nolint:gosec - rows, err := tx.QueryContext(ctx, query, m.tr.tableInfo.ID) + rows, err := tx.QueryContext( + ctx, + fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName), + m.tr.tableInfo.ID, + ) if err != nil { return errors.Trace(err) } @@ -282,7 +285,7 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64 if needAutoID && newRowIDBase == 0 && newStatus < metaStatusRestoreStarted { newStatus = metaStatusRestoreStarted } - query = fmt.Sprintf("update %s set row_id_base = ?, row_id_max = ?, status = ? where table_id = ? and task_id = ?", m.tableName) + query := fmt.Sprintf("update %s set row_id_base = ?, row_id_max = ?, status = ? where table_id = ? and task_id = ?", m.tableName) _, err := tx.ExecContext(ctx, query, newRowIDBase, newRowIDMax, newStatus.String(), m.tr.tableInfo.ID, m.taskID) if err != nil { return errors.Trace(err) @@ -381,9 +384,11 @@ func (m *dbTableMetaMgr) CheckAndUpdateLocalChecksum(ctx context.Context, checks needChecksum = true needRemoteDupe = true err = exec.Transact(ctx, "checksum pre-check", func(ctx context.Context, tx *sql.Tx) error { - // nolint:gosec - query := fmt.Sprintf("SELECT task_id, total_kvs_base, total_bytes_base, checksum_base, total_kvs, total_bytes, checksum, status, has_duplicates from %s WHERE table_id = ? FOR UPDATE", m.tableName) - rows, err := tx.QueryContext(ctx, query, m.tr.tableInfo.ID) + rows, err := tx.QueryContext( + ctx, + fmt.Sprintf("SELECT task_id, total_kvs_base, total_bytes_base, checksum_base, total_kvs, total_bytes, checksum, status, has_duplicates from %s WHERE table_id = ? FOR UPDATE", m.tableName), + m.tr.tableInfo.ID, + ) if err != nil { return errors.Annotate(err, "fetch task meta failed") } @@ -449,7 +454,7 @@ func (m *dbTableMetaMgr) CheckAndUpdateLocalChecksum(ctx context.Context, checks return errors.Trace(rows.Err()) } - query = fmt.Sprintf("update %s set total_kvs = ?, total_bytes = ?, checksum = ?, status = ?, has_duplicates = ? where table_id = ? and task_id = ?", m.tableName) + query := fmt.Sprintf("update %s set total_kvs = ?, total_bytes = ?, checksum = ?, status = ?, has_duplicates = ? where table_id = ? and task_id = ?", m.tableName) _, err = tx.ExecContext(ctx, query, checksum.SumKVS(), checksum.SumSize(), checksum.Sum(), newStatus.String(), hasLocalDupes, m.tr.tableInfo.ID, m.taskID) return errors.Annotate(err, "update local checksum failed") }) @@ -594,8 +599,10 @@ func (m *dbTaskMetaMgr) CheckTaskExist(ctx context.Context) (bool, error) { // avoid override existing metadata if the meta is already inserted. exist := false err := exec.Transact(ctx, "check whether this task has started before", func(ctx context.Context, tx *sql.Tx) error { - query := fmt.Sprintf("SELECT task_id from %s WHERE task_id = %d", m.tableName, m.taskID) // nolint:gosec - rows, err := tx.QueryContext(ctx, query) + rows, err := tx.QueryContext(ctx, + fmt.Sprintf("SELECT task_id from %s WHERE task_id = ?", m.tableName), + m.taskID, + ) if err != nil { return errors.Annotate(err, "fetch task meta failed") } @@ -636,8 +643,10 @@ func (m *dbTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(t return errors.Annotate(err, "enable pessimistic transaction failed") } return exec.Transact(ctx, "check tasks exclusively", func(ctx context.Context, tx *sql.Tx) error { - query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state, source_bytes, cluster_avail from %s FOR UPDATE", m.tableName) // nolint:gosec - rows, err := tx.QueryContext(ctx, query) + rows, err := tx.QueryContext( + ctx, + fmt.Sprintf("SELECT task_id, pd_cfgs, status, state, source_bytes, cluster_avail from %s FOR UPDATE", m.tableName), + ) if err != nil { return errors.Annotate(err, "fetch task metas failed") } @@ -696,8 +705,10 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U paused := false var pausedCfg storedCfgs err = exec.Transact(ctx, "check and pause schedulers", func(ctx context.Context, tx *sql.Tx) error { - query := fmt.Sprintf("SELECT task_id, pd_cfgs, status, state from %s FOR UPDATE", m.tableName) // nolint:gosec - rows, err := tx.QueryContext(ctx, query) + rows, err := tx.QueryContext( + ctx, + fmt.Sprintf("SELECT task_id, pd_cfgs, status, state from %s FOR UPDATE", m.tableName), + ) if err != nil { return errors.Annotate(err, "fetch task meta failed") } @@ -770,7 +781,7 @@ func (m *dbTaskMetaMgr) CheckAndPausePdSchedulers(ctx context.Context) (pdutil.U return errors.Trace(err) } - query = fmt.Sprintf("update %s set pd_cfgs = ?, status = ? where task_id = ?", m.tableName) + query := fmt.Sprintf("update %s set pd_cfgs = ?, status = ? where task_id = ?", m.tableName) _, err = tx.ExecContext(ctx, query, string(jsonByts), taskMetaStatusScheduleSet.String(), m.taskID) return errors.Annotate(err, "update task pd configs failed") @@ -822,8 +833,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool switchBack := true allFinished := finished err = exec.Transact(ctx, "check and finish schedulers", func(ctx context.Context, tx *sql.Tx) error { - query := fmt.Sprintf("SELECT task_id, status, state from %s FOR UPDATE", m.tableName) // nolint:gosec - rows, err := tx.QueryContext(ctx, query) + rows, err := tx.QueryContext(ctx, fmt.Sprintf("SELECT task_id, status, state from %s FOR UPDATE", m.tableName)) if err != nil { return errors.Annotate(err, "fetch task meta failed") } @@ -883,7 +893,7 @@ func (m *dbTaskMetaMgr) CheckAndFinishRestore(ctx context.Context, finished bool newStatus = taskMetaStatusSwitchSkipped } - query = fmt.Sprintf("update %s set status = ?, state = ? where task_id = ?", m.tableName) + query := fmt.Sprintf("update %s set status = ?, state = ? where task_id = ?", m.tableName) if _, err = tx.ExecContext(ctx, query, newStatus.String(), newState, m.taskID); err != nil { return errors.Trace(err) } diff --git a/br/pkg/lightning/restore/meta_manager_test.go b/br/pkg/lightning/restore/meta_manager_test.go index 31a3c35569c67..571a9d6cd5844 100644 --- a/br/pkg/lightning/restore/meta_manager_test.go +++ b/br/pkg/lightning/restore/meta_manager_test.go @@ -6,9 +6,9 @@ import ( "context" "database/sql/driver" "sort" + "testing" "github.com/DATA-DOG/go-sqlmock" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" @@ -18,26 +18,24 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" tmock "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" "go.uber.org/zap" ) -var _ = Suite(&metaMgrSuite{}) - type metaMgrSuite struct { mockDB sqlmock.Sqlmock - tr *TableRestore mgr *dbTableMetaMgr checksumMgr *testChecksumMgr } -func (s *metaMgrSuite) SetUpSuite(c *C) { +func newTableRestore(t *testing.T) *TableRestore { p := parser.New() se := tmock.NewContext() node, err := p.ParseOneStmt("CREATE TABLE `t1` (`c1` varchar(5) NOT NULL)", "utf8mb4", "utf8mb4_bin") - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), int64(1)) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo.State = model.StatePublic schema := "test" @@ -51,33 +49,36 @@ func (s *metaMgrSuite) SetUpSuite(c *C) { tableName := common.UniqueTable(schema, tb) logger := log.With(zap.String("table", tableName)) - s.tr = &TableRestore{ + return &TableRestore{ tableName: tableName, tableInfo: ti, logger: logger, } } -func (s *metaMgrSuite) SetUpTest(c *C) { +func newMetaMgrSuite(t *testing.T) (*metaMgrSuite, func()) { db, m, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + var s metaMgrSuite s.mgr = &dbTableMetaMgr{ session: db, taskID: 1, - tr: s.tr, + tr: newTableRestore(t), tableName: common.UniqueTable("test", TableMetaTableName), needChecksum: true, } s.mockDB = m s.checksumMgr = &testChecksumMgr{} + return &s, func() { + require.NoError(t, s.mockDB.ExpectationsWereMet()) + } } -func (s *metaMgrSuite) TearDownTest(c *C) { - c.Assert(s.mockDB.ExpectationsWereMet(), IsNil) -} +func TestAllocTableRowIDsSingleTable(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() -func (s *metaMgrSuite) TestAllocTableRowIDsSingleTable(c *C) { ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -88,13 +89,16 @@ func (s *metaMgrSuite) TestAllocTableRowIDsSingleTable(c *C) { s.prepareMock(rows, &nextID, updateArgs, nil, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(0)) - c.Assert(ck, IsNil) - c.Assert(s.checksumMgr.callCnt, Equals, 0) + require.NoError(t, err) + require.Equal(t, int64(0), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 0, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableAutoIDNot0(c *C) { +func TestAllocTableRowIDsSingleTableAutoIDNot0(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -106,13 +110,17 @@ func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableAutoIDNot0(c *C) { s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(998)) - c.Assert(ck, IsNil) - c.Assert(s.checksumMgr.callCnt, Equals, 1) + require.NoError(t, err) + require.Equal(t, int64(998), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 1, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableContainsData(c *C) { +func TestAllocTableRowIDsSingleTableContainsData(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() + ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -124,13 +132,16 @@ func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableContainsData(c *C) { s.prepareMock(rows, &nextID, updateArgs, &checksum, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(998)) - c.Assert(ck, DeepEquals, &checksum) - c.Assert(s.checksumMgr.callCnt, Equals, 1) + require.NoError(t, err) + require.Equal(t, int64(998), rowIDBase) + require.Equal(t, &checksum, ck) + require.Equal(t, 1, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableSkipChecksum(c *C) { +func TestAllocTableRowIDsSingleTableSkipChecksum(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() + s.mgr.needChecksum = false defer func() { s.mgr.needChecksum = true @@ -146,13 +157,17 @@ func (s *metaMgrSuite) TestAllocTableRowIDsSingleTableSkipChecksum(c *C) { s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(998)) - c.Assert(ck, IsNil) - c.Assert(s.checksumMgr.callCnt, Equals, 0) + require.NoError(t, err) + require.Equal(t, int64(998), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 0, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsAllocated(c *C) { +func TestAllocTableRowIDsAllocated(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() + ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -162,13 +177,16 @@ func (s *metaMgrSuite) TestAllocTableRowIDsAllocated(c *C) { s.prepareMock(rows, nil, nil, &checksum, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(998)) - c.Assert(ck, DeepEquals, &checksum) - c.Assert(s.checksumMgr.callCnt, Equals, 1) + require.NoError(t, err) + require.Equal(t, int64(998), rowIDBase) + require.Equal(t, &checksum, ck) + require.Equal(t, 1, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsFinished(c *C) { +func TestAllocTableRowIDsFinished(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() + ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -178,13 +196,15 @@ func (s *metaMgrSuite) TestAllocTableRowIDsFinished(c *C) { s.prepareMock(rows, nil, nil, nil, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(998)) - c.Assert(ck, DeepEquals, &checksum) - c.Assert(s.checksumMgr.callCnt, Equals, 0) + require.NoError(t, err) + require.Equal(t, int64(998), rowIDBase) + require.Equal(t, &checksum, ck) + require.Equal(t, 0, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsMultiTasksInit(c *C) { +func TestAllocTableRowIDsMultiTasksInit(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -196,13 +216,16 @@ func (s *metaMgrSuite) TestAllocTableRowIDsMultiTasksInit(c *C) { s.prepareMock(rows, &nextID, updateArgs, nil, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(0)) - c.Assert(ck, IsNil) - c.Assert(s.checksumMgr.callCnt, Equals, 0) + require.NoError(t, err) + require.Equal(t, int64(0), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 0, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) TestAllocTableRowIDsMultiTasksAllocated(c *C) { +func TestAllocTableRowIDsMultiTasksAllocated(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) rows := [][]driver.Value{ @@ -213,10 +236,11 @@ func (s *metaMgrSuite) TestAllocTableRowIDsMultiTasksAllocated(c *C) { s.prepareMock(rows, nil, updateArgs, nil, nil) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) - c.Assert(err, IsNil) - c.Assert(rowIDBase, Equals, int64(100)) - c.Assert(ck, IsNil) - c.Assert(s.checksumMgr.callCnt, Equals, 0) + require.NoError(t, err) + require.Equal(t, int64(100), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 0, s.checksumMgr.callCnt) } func (s *metaMgrSuite) prepareMock(rowsVal [][]driver.Value, nextRowID *int64, updateArgs []driver.Value, checksum *verification.KVChecksum, updateStatus *string) { @@ -264,26 +288,27 @@ func (s *metaMgrSuite) prepareMock(rowsVal [][]driver.Value, nextRowID *int64, u } } -var _ = Suite(&taskMetaMgrSuite{}) - type taskMetaMgrSuite struct { mgr *dbTaskMetaMgr mockDB sqlmock.Sqlmock } -func (s *taskMetaMgrSuite) SetUpTest(c *C) { +func newTaskMetaMgrSuite(t *testing.T) *taskMetaMgrSuite { db, m, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) + var s taskMetaMgrSuite s.mgr = &dbTaskMetaMgr{ session: db, taskID: 1, tableName: common.UniqueTable("test", "t1"), } s.mockDB = m + return &s } -func (s *taskMetaMgrSuite) TestCheckTasksExclusively(c *C) { +func TestCheckTasksExclusively(t *testing.T) { + s := newTaskMetaMgrSuite(t) s.mockDB.ExpectExec("SET SESSION tidb_txn_mode = 'pessimistic';"). WillReturnResult(sqlmock.NewResult(int64(0), int64(0))) s.mockDB.ExpectBegin() @@ -304,12 +329,12 @@ func (s *taskMetaMgrSuite) TestCheckTasksExclusively(c *C) { s.mockDB.ExpectCommit() err := s.mgr.CheckTasksExclusively(context.Background(), func(tasks []taskMeta) ([]taskMeta, error) { - c.Assert(len(tasks), Equals, 5) + require.Equal(t, 5, len(tasks)) sort.Slice(tasks, func(i, j int) bool { return tasks[i].taskID < tasks[j].taskID }) for j := 0; j < 5; j++ { - c.Assert(tasks[j].taskID, Equals, int64(j)) + require.Equal(t, int64(j), tasks[j].taskID) } var newTasks []taskMeta @@ -320,6 +345,16 @@ func (s *taskMetaMgrSuite) TestCheckTasksExclusively(c *C) { } return newTasks, nil }) - c.Assert(err, IsNil) + require.NoError(t, err) + +} + +type testChecksumMgr struct { + checksum RemoteChecksum + callCnt int +} +func (t *testChecksumMgr) Checksum(ctx context.Context, tableInfo *checkpoints.TidbTableInfo) (*RemoteChecksum, error) { + t.callCnt++ + return &t.checksum, nil } diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 1333c6c7227b5..df888dd81bd24 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1467,6 +1467,9 @@ func (rc *Controller) restoreTables(ctx context.Context) (finalErr error) { if err != nil { return errors.Trace(err) } + if cp.Status < checkpoints.CheckpointStatusAllWritten && len(tableMeta.DataFiles) == 0 { + continue + } igCols, err := rc.cfg.Mydumper.IgnoreColumns.GetIgnoreColumns(dbInfo.Name, tableInfo.Name, rc.cfg.Mydumper.CaseSensitive) if err != nil { return errors.Trace(err) @@ -1508,8 +1511,8 @@ func (rc *Controller) restoreTables(ctx context.Context) (finalErr error) { for task := range postProcessTaskChan { metaMgr := rc.metaMgrBuilder.TableMetaMgr(task.tr) // force all the remain post-process tasks to be executed - _, err = task.tr.postProcess(ctx2, rc, task.cp, true, metaMgr) - restoreErr.Set(err) + _, err2 := task.tr.postProcess(ctx2, rc, task.cp, true, metaMgr) + restoreErr.Set(err2) } }() } @@ -1526,7 +1529,6 @@ func (tr *TableRestore) restoreTable( cp *checkpoints.TableCheckpoint, ) (bool, error) { // 1. Load the table info. - select { case <-ctx.Done(): return false, ctx.Err() diff --git a/br/pkg/lightning/restore/restore_schema_test.go b/br/pkg/lightning/restore/restore_schema_test.go new file mode 100644 index 0000000000000..e0a9a9d8a43da --- /dev/null +++ b/br/pkg/lightning/restore/restore_schema_test.go @@ -0,0 +1,276 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package restore + +import ( + "context" + stderrors "errors" + "fmt" + "testing" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/golang/mock/gomock" + "github.com/pingcap/errors" + filter "github.com/pingcap/tidb-tools/pkg/table-filter" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" + + "github.com/pingcap/tidb/br/pkg/lightning/backend" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/mock" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + tmock "github.com/pingcap/tidb/util/mock" +) + +type restoreSchemaSuite struct { + suite.Suite + ctx context.Context + rc *Controller + controller *gomock.Controller + tableInfos []*model.TableInfo +} + +func TestRestoreSchemaSuite(t *testing.T) { + suite.Run(t, new(restoreSchemaSuite)) +} + +func (s *restoreSchemaSuite) SetupSuite() { + ctx := context.Background() + fakeDataDir := s.T().TempDir() + + store, err := storage.NewLocalStorage(fakeDataDir) + require.NoError(s.T(), err) + // restore database schema file + fakeDBName := "fakedb" + // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}-schema-create.sql' + fakeFileName := fmt.Sprintf("%s-schema-create.sql", fakeDBName) + err = store.WriteFile(ctx, fakeFileName, []byte(fmt.Sprintf("CREATE DATABASE %s;", fakeDBName))) + require.NoError(s.T(), err) + // restore table schema files + fakeTableFilesCount := 8 + + p := parser.New() + p.SetSQLMode(mysql.ModeANSIQuotes) + se := tmock.NewContext() + + tableInfos := make([]*model.TableInfo, 0, fakeTableFilesCount) + for i := 1; i <= fakeTableFilesCount; i++ { + fakeTableName := fmt.Sprintf("tbl%d", i) + // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}.{table}-schema.sql' + fakeFileName := fmt.Sprintf("%s.%s-schema.sql", fakeDBName, fakeTableName) + fakeFileContent := fmt.Sprintf("CREATE TABLE %s(i TINYINT);", fakeTableName) + err = store.WriteFile(ctx, fakeFileName, []byte(fakeFileContent)) + require.NoError(s.T(), err) + + node, err := p.ParseOneStmt(fakeFileContent, "", "") + require.NoError(s.T(), err) + core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) + require.NoError(s.T(), err) + core.State = model.StatePublic + tableInfos = append(tableInfos, core) + } + s.tableInfos = tableInfos + // restore view schema files + fakeViewFilesCount := 8 + for i := 1; i <= fakeViewFilesCount; i++ { + fakeViewName := fmt.Sprintf("tbl%d", i) + // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}.{table}-schema-view.sql' + fakeFileName := fmt.Sprintf("%s.%s-schema-view.sql", fakeDBName, fakeViewName) + fakeFileContent := []byte(fmt.Sprintf("CREATE ALGORITHM=UNDEFINED VIEW `%s` (`i`) AS SELECT `i` FROM `%s`.`%s`;", fakeViewName, fakeDBName, fmt.Sprintf("tbl%d", i))) + err = store.WriteFile(ctx, fakeFileName, fakeFileContent) + require.NoError(s.T(), err) + } + config := config.NewConfig() + config.Mydumper.DefaultFileRules = true + config.Mydumper.CharacterSet = "utf8mb4" + config.App.RegionConcurrency = 8 + mydumpLoader, err := mydump.NewMyDumpLoaderWithStore(ctx, config, store) + require.NoError(s.T(), err) + s.rc = &Controller{ + checkTemplate: NewSimpleTemplate(), + cfg: config, + store: store, + dbMetas: mydumpLoader.GetDatabases(), + checkpointsDB: &checkpoints.NullCheckpointsDB{}, + } +} + +//nolint:interfacer // change test case signature might cause Check failed to find this test case? +func (s *restoreSchemaSuite) SetupTest() { + s.controller, s.ctx = gomock.WithContext(context.Background(), s.T()) + mockBackend := mock.NewMockBackend(s.controller) + mockBackend.EXPECT(). + FetchRemoteTableModels(gomock.Any(), gomock.Any()). + AnyTimes(). + Return(s.tableInfos, nil) + mockBackend.EXPECT().Close() + s.rc.backend = backend.MakeBackend(mockBackend) + + mockDB, sqlMock, err := sqlmock.New() + require.NoError(s.T(), err) + for i := 0; i < 17; i++ { + sqlMock.ExpectExec(".*").WillReturnResult(sqlmock.NewResult(int64(i), 1)) + } + mockTiDBGlue := mock.NewMockGlue(s.controller) + mockTiDBGlue.EXPECT().GetDB().AnyTimes().Return(mockDB, nil) + mockTiDBGlue.EXPECT(). + OwnsSQLExecutor(). + AnyTimes(). + Return(true) + parser := parser.New() + mockTiDBGlue.EXPECT(). + GetParser(). + AnyTimes(). + Return(parser) + s.rc.tidbGlue = mockTiDBGlue +} + +func (s *restoreSchemaSuite) TearDownTest() { + exec := mock.NewMockSQLExecutor(s.controller) + exec.EXPECT().Close() + mockTiDBGlue := mock.NewMockGlue(s.controller) + mockTiDBGlue.EXPECT(). + GetSQLExecutor(). + AnyTimes(). + Return(exec) + s.rc.tidbGlue = mockTiDBGlue + + s.rc.Close() + s.controller.Finish() +} + +func (s *restoreSchemaSuite) TestRestoreSchemaSuccessful() { + // before restore, if sysVars is initialized by other test, the time_zone should be default value + if len(s.rc.sysVars) > 0 { + tz, ok := s.rc.sysVars["time_zone"] + require.True(s.T(), ok) + require.Equal(s.T(), "SYSTEM", tz) + } + + exec := mock.NewMockSQLExecutor(s.controller) + exec.EXPECT(). + QueryStringsWithLog(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()). + AnyTimes(). + Return([][]string{{"time_zone", "SYSTEM"}}, nil) + mockTiDBGlue := s.rc.tidbGlue.(*mock.MockGlue) + mockTiDBGlue.EXPECT(). + GetSQLExecutor(). + AnyTimes(). + Return(exec) + + s.rc.cfg.TiDB.Vars = map[string]string{ + "time_zone": "UTC", + } + err := s.rc.restoreSchema(s.ctx) + require.NoError(s.T(), err) + + // test after restore schema, sysVars has been updated + tz, ok := s.rc.sysVars["time_zone"] + require.True(s.T(), ok) + require.Equal(s.T(), "UTC", tz) +} + +func (s *restoreSchemaSuite) TestRestoreSchemaFailed() { + // use injectErr which cannot be retried + injectErr := stderrors.New("could not match actual sql") + mockDB, sqlMock, err := sqlmock.New() + require.NoError(s.T(), err) + sqlMock.ExpectExec(".*").WillReturnError(injectErr) + for i := 0; i < 16; i++ { + sqlMock.ExpectExec(".*").WillReturnResult(sqlmock.NewResult(int64(i), 1)) + } + + mockTiDBGlue := mock.NewMockGlue(s.controller) + mockTiDBGlue.EXPECT(). + GetDB(). + AnyTimes(). + Return(mockDB, nil) + mockTiDBGlue.EXPECT(). + OwnsSQLExecutor(). + AnyTimes(). + Return(true) + parser := parser.New() + mockTiDBGlue.EXPECT(). + GetParser(). + AnyTimes(). + Return(parser) + s.rc.tidbGlue = mockTiDBGlue + err = s.rc.restoreSchema(s.ctx) + require.Error(s.T(), err) + require.True(s.T(), errors.ErrorEqual(err, injectErr)) +} + +// When restoring a CSV with `-no-schema` and the target table doesn't exist +// then we can't restore the schema as the `Path` is empty. This is to make +// sure this results in the correct error. +// https://github.com/pingcap/br/issues/1394 +func (s *restoreSchemaSuite) TestNoSchemaPath() { + fakeTable := mydump.MDTableMeta{ + DB: "fakedb", + Name: "fake1", + SchemaFile: mydump.FileInfo{ + TableName: filter.Table{ + Schema: "fakedb", + Name: "fake1", + }, + FileMeta: mydump.SourceFileMeta{ + Path: "", + }, + }, + DataFiles: []mydump.FileInfo{}, + TotalSize: 0, + } + s.rc.dbMetas[0].Tables = append(s.rc.dbMetas[0].Tables, &fakeTable) + err := s.rc.restoreSchema(s.ctx) + require.Error(s.T(), err) + require.Regexp(s.T(), `table .* schema not found`, err.Error()) + s.rc.dbMetas[0].Tables = s.rc.dbMetas[0].Tables[:len(s.rc.dbMetas[0].Tables)-1] +} + +func (s *restoreSchemaSuite) TestRestoreSchemaContextCancel() { + childCtx, cancel := context.WithCancel(s.ctx) + mockDB, sqlMock, err := sqlmock.New() + require.NoError(s.T(), err) + for i := 0; i < 17; i++ { + sqlMock.ExpectExec(".*").WillReturnResult(sqlmock.NewResult(int64(i), 1)) + } + mockTiDBGlue := mock.NewMockGlue(s.controller) + mockTiDBGlue.EXPECT(). + GetDB(). + AnyTimes(). + Do(func() { cancel() }). + Return(mockDB, nil) + mockTiDBGlue.EXPECT(). + OwnsSQLExecutor(). + AnyTimes(). + Return(true) + parser := parser.New() + mockTiDBGlue.EXPECT(). + GetParser(). + AnyTimes(). + Return(parser) + s.rc.tidbGlue = mockTiDBGlue + err = s.rc.restoreSchema(childCtx) + cancel() + require.Error(s.T(), err) + require.Equal(s.T(), childCtx.Err(), err) +} diff --git a/br/pkg/lightning/restore/restore_test.go b/br/pkg/lightning/restore/restore_test.go index 422fc03d5305b..3daa8b696dc19 100644 --- a/br/pkg/lightning/restore/restore_test.go +++ b/br/pkg/lightning/restore/restore_test.go @@ -16,64 +16,32 @@ package restore import ( "context" - "encoding/json" "fmt" - "net/http" - "net/http/httptest" - "os" "path/filepath" "sort" - "strconv" - "strings" "sync/atomic" + "testing" "time" - "unicode/utf8" "github.com/DATA-DOG/go-sqlmock" - "github.com/docker/go-units" - "github.com/golang/mock/gomock" - "github.com/google/uuid" - . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/import_kvpb" - "github.com/pingcap/kvproto/pkg/metapb" - filter "github.com/pingcap/tidb-tools/pkg/table-filter" - "github.com/pingcap/tidb/br/pkg/lightning/backend" - "github.com/pingcap/tidb/br/pkg/lightning/backend/importer" - "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" - "github.com/pingcap/tidb/br/pkg/lightning/backend/noop" - "github.com/pingcap/tidb/br/pkg/lightning/backend/tidb" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/errormanager" "github.com/pingcap/tidb/br/pkg/lightning/glue" "github.com/pingcap/tidb/br/pkg/lightning/log" - "github.com/pingcap/tidb/br/pkg/lightning/metric" - "github.com/pingcap/tidb/br/pkg/lightning/mydump" - "github.com/pingcap/tidb/br/pkg/lightning/verification" - "github.com/pingcap/tidb/br/pkg/lightning/web" - "github.com/pingcap/tidb/br/pkg/lightning/worker" - "github.com/pingcap/tidb/br/pkg/mock" - "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/version/build" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/types" - "github.com/pingcap/tidb/table/tables" tmock "github.com/pingcap/tidb/util/mock" - "github.com/tikv/pd/server/api" + "github.com/stretchr/testify/require" ) -var _ = Suite(&restoreSuite{}) - -type restoreSuite struct{} - -func (s *restoreSuite) TestNewTableRestore(c *C) { +func TestNewTableRestore(t *testing.T) { testCases := []struct { name string createStmt string @@ -89,9 +57,9 @@ func (s *restoreSuite) TestNewTableRestore(c *C) { dbInfo := &checkpoints.TidbDBInfo{Name: "mockdb", Tables: map[string]*checkpoints.TidbTableInfo{}} for i, tc := range testCases { node, err := p.ParseOneStmt(tc.createStmt, "utf8mb4", "utf8mb4_bin") - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), int64(i+1)) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo.State = model.StatePublic dbInfo.Tables[tc.name] = &checkpoints.TidbTableInfo{ @@ -104,12 +72,12 @@ func (s *restoreSuite) TestNewTableRestore(c *C) { tableInfo := dbInfo.Tables[tc.name] tableName := common.UniqueTable("mockdb", tableInfo.Name) tr, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) - c.Assert(tr, NotNil) - c.Assert(err, IsNil) + require.NotNil(t, tr) + require.NoError(t, err) } } -func (s *restoreSuite) TestNewTableRestoreFailure(c *C) { +func TestNewTableRestoreFailure(t *testing.T) { tableInfo := &checkpoints.TidbTableInfo{ Name: "failure", Core: &model.TableInfo{}, @@ -120,10 +88,10 @@ func (s *restoreSuite) TestNewTableRestoreFailure(c *C) { tableName := common.UniqueTable("mockdb", "failure") _, err := NewTableRestore(tableName, nil, dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) - c.Assert(err, ErrorMatches, `failed to tables\.TableFromMeta.*`) + require.Regexp(t, `failed to tables\.TableFromMeta.*`, err.Error()) } -func (s *restoreSuite) TestErrorSummaries(c *C) { +func TestErrorSummaries(t *testing.T) { logger, buffer := log.MakeTestLogger() es := makeErrorSummaries(logger) @@ -133,15 +101,16 @@ func (s *restoreSuite) TestErrorSummaries(c *C) { lines := buffer.Lines() sort.Strings(lines[1:]) - c.Assert(lines, DeepEquals, []string{ + require.Equal(t, []string{ `{"$lvl":"ERROR","$msg":"tables failed to be imported","count":2}`, `{"$lvl":"ERROR","$msg":"-","table":"first","status":"analyzed","error":"a1 error"}`, `{"$lvl":"ERROR","$msg":"-","table":"second","status":"written","error":"b2 error"}`, - }) + }, lines) } -func (s *restoreSuite) TestVerifyCheckpoint(c *C) { - dir := c.MkDir() +func TestVerifyCheckpoint(t *testing.T) { + dir := t.TempDir() + cpdb := checkpoints.NewFileCheckpointsDB(filepath.Join(dir, "cp.pb")) defer cpdb.Close() ctx := context.Background() @@ -152,8 +121,8 @@ func (s *restoreSuite) TestVerifyCheckpoint(c *C) { }() taskCp, err := cpdb.TaskCheckpoint(ctx) - c.Assert(err, IsNil) - c.Assert(taskCp, IsNil) + require.NoError(t, err) + require.Nil(t, taskCp) newCfg := func() *config.Config { cfg := config.NewConfig() @@ -169,7 +138,7 @@ func (s *restoreSuite) TestVerifyCheckpoint(c *C) { } err = cpdb.Initialize(ctx, newCfg(), map[string]*checkpoints.TidbDBInfo{}) - c.Assert(err, IsNil) + require.NoError(t, err) adjustFuncs := map[string]func(cfg *config.Config){ "tikv-importer.backend": func(cfg *config.Config) { @@ -197,16 +166,16 @@ func (s *restoreSuite) TestVerifyCheckpoint(c *C) { // default mode, will return error taskCp, err = cpdb.TaskCheckpoint(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) for conf, fn := range adjustFuncs { cfg := newCfg() fn(cfg) err := verifyCheckpoint(cfg, taskCp) if conf == "version" { build.ReleaseVersion = actualReleaseVersion - c.Assert(err, ErrorMatches, "lightning version is 'some newer version', but checkpoint was created at '"+actualReleaseVersion+"'.*") + require.Regexp(t, "lightning version is 'some newer version', but checkpoint was created at '"+actualReleaseVersion+"'.*", err.Error()) } else { - c.Assert(err, ErrorMatches, fmt.Sprintf("config '%s' value '.*' different from checkpoint value .*", conf)) + require.Regexp(t, fmt.Sprintf("config '%s' value '.*' different from checkpoint value .*", conf), err.Error()) } } @@ -218,18 +187,18 @@ func (s *restoreSuite) TestVerifyCheckpoint(c *C) { cfg.App.CheckRequirements = false fn(cfg) err := cpdb.Initialize(context.Background(), cfg, map[string]*checkpoints.TidbDBInfo{}) - c.Assert(err, IsNil) + require.NoError(t, err) } } -func (s *restoreSuite) TestDiskQuotaLock(c *C) { +func TestDiskQuotaLock(t *testing.T) { lock := newDiskQuotaLock() lock.Lock() - c.Assert(lock.TryRLock(), IsFalse) + require.False(t, lock.TryRLock()) lock.Unlock() - c.Assert(lock.TryRLock(), IsTrue) - c.Assert(lock.TryRLock(), IsTrue) + require.True(t, lock.TryRLock()) + require.True(t, lock.TryRLock()) rLocked := 2 lockHeld := make(chan struct{}) @@ -243,7 +212,7 @@ func (s *restoreSuite) TestDiskQuotaLock(c *C) { } select { case <-lockHeld: - c.Fatal("write lock is held before all read locks are released") + t.Fatal("write lock is held before all read locks are released") case <-time.NewTimer(10 * time.Millisecond).C: } for ; rLocked > 0; rLocked-- { @@ -318,13 +287,13 @@ func (cp panicCheckpointDB) Initialize(context.Context, *config.Config, map[stri panic("should not reach here") } -func (s *restoreSuite) TestPreCheckFailed(c *C) { +func TestPreCheckFailed(t *testing.T) { cfg := config.NewConfig() cfg.TikvImporter.Backend = config.BackendTiDB cfg.App.CheckRequirements = false db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) g := glue.NewExternalTiDBGlue(db, mysql.ModeNone) ctl := &Controller{ @@ -344,8 +313,8 @@ func (s *restoreSuite) TestPreCheckFailed(c *C) { mock.ExpectCommit() // precheck failed, will not do init checkpoint. err = ctl.Run(context.Background()) - c.Assert(err, ErrorMatches, ".*mock init meta failure") - c.Assert(mock.ExpectationsWereMet(), IsNil) + require.Regexp(t, ".*mock init meta failure", err.Error()) + require.NoError(t, mock.ExpectationsWereMet()) mock.ExpectBegin() mock.ExpectQuery("SHOW VARIABLES WHERE Variable_name IN .*"). @@ -355,2494 +324,6 @@ func (s *restoreSuite) TestPreCheckFailed(c *C) { ctl.saveCpCh = make(chan saveCp) // precheck failed, will not do init checkpoint. err1 := ctl.Run(context.Background()) - c.Assert(err1.Error(), Equals, err.Error()) - c.Assert(mock.ExpectationsWereMet(), IsNil) -} - -var _ = Suite(&tableRestoreSuite{}) - -type tableRestoreSuiteBase struct { - tr *TableRestore - cfg *config.Config - - tableInfo *checkpoints.TidbTableInfo - dbInfo *checkpoints.TidbDBInfo - tableMeta *mydump.MDTableMeta - - store storage.ExternalStorage -} - -type tableRestoreSuite struct { - tableRestoreSuiteBase -} - -func (s *tableRestoreSuiteBase) SetUpSuite(c *C) { - // Produce a mock table info - - p := parser.New() - p.SetSQLMode(mysql.ModeANSIQuotes) - se := tmock.NewContext() - node, err := p.ParseOneStmt(` - CREATE TABLE "table" ( - a INT, - b INT, - c INT, - KEY (b) - ) - `, "", "") - c.Assert(err, IsNil) - core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) - c.Assert(err, IsNil) - core.State = model.StatePublic - - s.tableInfo = &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} - s.dbInfo = &checkpoints.TidbDBInfo{ - Name: "db", - Tables: map[string]*checkpoints.TidbTableInfo{"table": s.tableInfo}, - } - - // Write some sample SQL dump - - fakeDataDir := c.MkDir() - - store, err := storage.NewLocalStorage(fakeDataDir) - c.Assert(err, IsNil) - s.store = store - - fakeDataFilesCount := 6 - fakeDataFilesContent := []byte("INSERT INTO `table` VALUES (1, 2, 3);") - c.Assert(len(fakeDataFilesContent), Equals, 37) - fakeDataFiles := make([]mydump.FileInfo, 0, fakeDataFilesCount) - for i := 1; i <= fakeDataFilesCount; i++ { - fakeFileName := fmt.Sprintf("db.table.%d.sql", i) - fakeDataPath := filepath.Join(fakeDataDir, fakeFileName) - err = os.WriteFile(fakeDataPath, fakeDataFilesContent, 0o644) - c.Assert(err, IsNil) - fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ - TableName: filter.Table{Schema: "db", Name: "table"}, - FileMeta: mydump.SourceFileMeta{ - Path: fakeFileName, - Type: mydump.SourceTypeSQL, - SortKey: strconv.Itoa(i), - FileSize: 37, - }, - }) - } - - fakeCsvContent := []byte("1,2,3\r\n4,5,6\r\n") - csvName := "db.table.99.csv" - err = os.WriteFile(filepath.Join(fakeDataDir, csvName), fakeCsvContent, 0o644) - c.Assert(err, IsNil) - fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ - TableName: filter.Table{Schema: "db", Name: "table"}, - FileMeta: mydump.SourceFileMeta{ - Path: csvName, - Type: mydump.SourceTypeCSV, - SortKey: "99", - FileSize: 14, - }, - }) - - s.tableMeta = &mydump.MDTableMeta{ - DB: "db", - Name: "table", - TotalSize: 222, - SchemaFile: mydump.FileInfo{ - TableName: filter.Table{Schema: "db", Name: "table"}, - FileMeta: mydump.SourceFileMeta{ - Path: "db.table-schema.sql", - Type: mydump.SourceTypeTableSchema, - }, - }, - DataFiles: fakeDataFiles, - } -} - -func (s *tableRestoreSuiteBase) SetUpTest(c *C) { - // Collect into the test TableRestore structure - var err error - s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) - c.Assert(err, IsNil) - - s.cfg = config.NewConfig() - s.cfg.Mydumper.BatchSize = 111 - s.cfg.App.TableConcurrency = 2 -} - -func (s *tableRestoreSuite) TestPopulateChunks(c *C) { - _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp", "return(1234567897)") - defer func() { - _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp") - }() - - cp := &checkpoints.TableCheckpoint{ - Engines: make(map[int32]*checkpoints.EngineCheckpoint), - } - - rc := &Controller{cfg: s.cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: s.store} - err := s.tr.populateChunks(context.Background(), rc, cp) - c.Assert(err, IsNil) - //nolint:dupl // false positive. - c.Assert(cp.Engines, DeepEquals, map[int32]*checkpoints.EngineCheckpoint{ - -1: { - Status: checkpoints.CheckpointStatusLoaded, - }, - 0: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[0].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[0].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 0, - RowIDMax: 7, // 37 bytes with 3 columns can store at most 7 rows. - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[1].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 7, - RowIDMax: 14, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[2].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[2].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 14, - RowIDMax: 21, - }, - Timestamp: 1234567897, - }, - }, - }, - 1: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[3].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[3].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 21, - RowIDMax: 28, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[4].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[4].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 28, - RowIDMax: 35, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[5].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[5].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 35, - RowIDMax: 42, - }, - Timestamp: 1234567897, - }, - }, - }, - 2: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[6].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[6].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 14, - PrevRowIDMax: 42, - RowIDMax: 46, - }, - Timestamp: 1234567897, - }, - }, - }, - }) - - // set csv header to true, this will cause check columns fail - s.cfg.Mydumper.CSV.Header = true - s.cfg.Mydumper.StrictFormat = true - regionSize := s.cfg.Mydumper.MaxRegionSize - s.cfg.Mydumper.MaxRegionSize = 5 - err = s.tr.populateChunks(context.Background(), rc, cp) - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, `.*unknown columns in header \[1 2 3\]`) - s.cfg.Mydumper.MaxRegionSize = regionSize - s.cfg.Mydumper.CSV.Header = false -} - -type errorLocalWriter struct{} - -func (w errorLocalWriter) AppendRows(context.Context, string, []string, kv.Rows) error { - return errors.New("mock write rows failed") -} - -func (w errorLocalWriter) IsSynced() bool { - return true -} - -func (w errorLocalWriter) Close(context.Context) (backend.ChunkFlushStatus, error) { - return nil, nil -} - -func (s *tableRestoreSuite) TestRestoreEngineFailed(c *C) { - ctx := context.Background() - ctrl := gomock.NewController(c) - mockBackend := mock.NewMockBackend(ctrl) - rc := &Controller{ - cfg: s.cfg, - pauser: DeliverPauser, - ioWorkers: worker.NewPool(ctx, 1, "io"), - regionWorkers: worker.NewPool(ctx, 10, "region"), - store: s.store, - backend: backend.MakeBackend(mockBackend), - errorSummaries: makeErrorSummaries(log.L()), - saveCpCh: make(chan saveCp, 1), - diskQuotaLock: newDiskQuotaLock(), - } - defer close(rc.saveCpCh) - go func() { - for cp := range rc.saveCpCh { - cp.waitCh <- nil - } - }() - - cp := &checkpoints.TableCheckpoint{ - Engines: make(map[int32]*checkpoints.EngineCheckpoint), - } - err := s.tr.populateChunks(ctx, rc, cp) - c.Assert(err, IsNil) - - tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), s.tableInfo.Core) - c.Assert(err, IsNil) - _, indexUUID := backend.MakeUUID("`db`.`table`", -1) - _, dataUUID := backend.MakeUUID("`db`.`table`", 0) - realBackend := tidb.NewTiDBBackend(nil, "replace", nil) - mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - mockBackend.EXPECT().CloseEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()). - Return(realBackend.NewEncoder(tbl, &kv.SessionOptions{})). - AnyTimes() - mockBackend.EXPECT().MakeEmptyRows().Return(realBackend.MakeEmptyRows()).AnyTimes() - mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), dataUUID).Return(noop.Writer{}, nil) - mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), indexUUID). - Return(nil, errors.New("mock open index local writer failed")) - openedIdxEngine, err := rc.backend.OpenEngine(ctx, nil, "`db`.`table`", -1) - c.Assert(err, IsNil) - - // open the first engine meet error, should directly return the error - _, err = s.tr.restoreEngine(ctx, rc, openedIdxEngine, 0, cp.Engines[0]) - c.Assert(err, ErrorMatches, "mock open index local writer failed") - - localWriter := func(ctx context.Context, cfg *backend.LocalWriterConfig, engineUUID uuid.UUID) (backend.EngineWriter, error) { - time.Sleep(20 * time.Millisecond) - select { - case <-ctx.Done(): - return nil, errors.New("mock open index local writer failed after ctx.Done") - default: - return noop.Writer{}, nil - } - } - mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), dataUUID).Return(errorLocalWriter{}, nil).AnyTimes() - mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), indexUUID). - DoAndReturn(localWriter).AnyTimes() - - openedIdxEngine, err = rc.backend.OpenEngine(ctx, nil, "`db`.`table`", -1) - c.Assert(err, IsNil) - - // open engine failed after write rows failed, should return write rows error - _, err = s.tr.restoreEngine(ctx, rc, openedIdxEngine, 0, cp.Engines[0]) - c.Assert(err, ErrorMatches, "mock write rows failed") -} - -func (s *tableRestoreSuite) TestPopulateChunksCSVHeader(c *C) { - fakeDataDir := c.MkDir() - store, err := storage.NewLocalStorage(fakeDataDir) - c.Assert(err, IsNil) - - fakeDataFiles := make([]mydump.FileInfo, 0) - - fakeCsvContents := []string{ - // small full header - "a,b,c\r\n1,2,3\r\n", - // small partial header - "b,c\r\n2,3\r\n", - // big full header - "a,b,c\r\n90000,80000,700000\r\n1000,2000,3000\r\n11,22,33\r\n3,4,5\r\n", - // big full header unordered - "c,a,b\r\n,1000,2000,3000\r\n11,22,33\r\n1000,2000,404\r\n3,4,5\r\n90000,80000,700000\r\n7999999,89999999,9999999\r\n", - // big partial header - "b,c\r\n2000001,30000001\r\n35231616,462424626\r\n62432,434898934\r\n", - } - total := 0 - for i, s := range fakeCsvContents { - csvName := fmt.Sprintf("db.table.%02d.csv", i) - err := os.WriteFile(filepath.Join(fakeDataDir, csvName), []byte(s), 0o644) - c.Assert(err, IsNil) - fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ - TableName: filter.Table{Schema: "db", Name: "table"}, - FileMeta: mydump.SourceFileMeta{Path: csvName, Type: mydump.SourceTypeCSV, SortKey: fmt.Sprintf("%02d", i), FileSize: int64(len(s))}, - }) - total += len(s) - } - tableMeta := &mydump.MDTableMeta{ - DB: "db", - Name: "table", - TotalSize: int64(total), - SchemaFile: mydump.FileInfo{TableName: filter.Table{Schema: "db", Name: "table"}, FileMeta: mydump.SourceFileMeta{Path: "db.table-schema.sql", Type: mydump.SourceTypeTableSchema}}, - DataFiles: fakeDataFiles, - } - - _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp", "return(1234567897)") - defer func() { - _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp") - }() - - cp := &checkpoints.TableCheckpoint{ - Engines: make(map[int32]*checkpoints.EngineCheckpoint), - } - - cfg := config.NewConfig() - cfg.Mydumper.BatchSize = 100 - cfg.Mydumper.MaxRegionSize = 40 - - cfg.Mydumper.CSV.Header = true - cfg.Mydumper.StrictFormat = true - rc := &Controller{cfg: cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: store} - - tr, err := NewTableRestore("`db`.`table`", tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) - c.Assert(err, IsNil) - c.Assert(tr.populateChunks(context.Background(), rc, cp), IsNil) - - c.Assert(cp.Engines, DeepEquals, map[int32]*checkpoints.EngineCheckpoint{ - -1: { - Status: checkpoints.CheckpointStatusLoaded, - }, - 0: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[0].FileMeta.Path, Offset: 0}, - FileMeta: tableMeta.DataFiles[0].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 14, - PrevRowIDMax: 0, - RowIDMax: 4, // 37 bytes with 3 columns can store at most 7 rows. - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, - FileMeta: tableMeta.DataFiles[1].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 10, - PrevRowIDMax: 4, - RowIDMax: 7, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[2].FileMeta.Path, Offset: 6}, - FileMeta: tableMeta.DataFiles[2].FileMeta, - ColumnPermutation: []int{0, 1, 2, -1}, - Chunk: mydump.Chunk{ - Offset: 6, - EndOffset: 52, - PrevRowIDMax: 7, - RowIDMax: 20, - Columns: []string{"a", "b", "c"}, - }, - - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[2].FileMeta.Path, Offset: 52}, - FileMeta: tableMeta.DataFiles[2].FileMeta, - ColumnPermutation: []int{0, 1, 2, -1}, - Chunk: mydump.Chunk{ - Offset: 52, - EndOffset: 60, - PrevRowIDMax: 20, - RowIDMax: 22, - Columns: []string{"a", "b", "c"}, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 6}, - FileMeta: tableMeta.DataFiles[3].FileMeta, - ColumnPermutation: []int{1, 2, 0, -1}, - Chunk: mydump.Chunk{ - Offset: 6, - EndOffset: 48, - PrevRowIDMax: 22, - RowIDMax: 35, - Columns: []string{"c", "a", "b"}, - }, - Timestamp: 1234567897, - }, - }, - }, - 1: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 48}, - FileMeta: tableMeta.DataFiles[3].FileMeta, - ColumnPermutation: []int{1, 2, 0, -1}, - Chunk: mydump.Chunk{ - Offset: 48, - EndOffset: 101, - PrevRowIDMax: 35, - RowIDMax: 48, - Columns: []string{"c", "a", "b"}, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 101}, - FileMeta: tableMeta.DataFiles[3].FileMeta, - ColumnPermutation: []int{1, 2, 0, -1}, - Chunk: mydump.Chunk{ - Offset: 101, - EndOffset: 102, - PrevRowIDMax: 48, - RowIDMax: 48, - Columns: []string{"c", "a", "b"}, - }, - Timestamp: 1234567897, - }, - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[4].FileMeta.Path, Offset: 4}, - FileMeta: tableMeta.DataFiles[4].FileMeta, - ColumnPermutation: []int{-1, 0, 1, -1}, - Chunk: mydump.Chunk{ - Offset: 4, - EndOffset: 59, - PrevRowIDMax: 48, - RowIDMax: 61, - Columns: []string{"b", "c"}, - }, - Timestamp: 1234567897, - }, - }, - }, - 2: { - Status: checkpoints.CheckpointStatusLoaded, - Chunks: []*checkpoints.ChunkCheckpoint{ - { - Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[4].FileMeta.Path, Offset: 59}, - FileMeta: tableMeta.DataFiles[4].FileMeta, - ColumnPermutation: []int{-1, 0, 1, -1}, - Chunk: mydump.Chunk{ - Offset: 59, - EndOffset: 60, - PrevRowIDMax: 61, - RowIDMax: 61, - Columns: []string{"b", "c"}, - }, - Timestamp: 1234567897, - }, - }, - }, - }) -} - -func (s *tableRestoreSuite) TestGetColumnsNames(c *C) { - c.Assert(getColumnNames(s.tableInfo.Core, []int{0, 1, 2, -1}), DeepEquals, []string{"a", "b", "c"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{1, 0, 2, -1}), DeepEquals, []string{"b", "a", "c"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{-1, 0, 1, -1}), DeepEquals, []string{"b", "c"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{0, 1, -1, -1}), DeepEquals, []string{"a", "b"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{1, -1, 0, -1}), DeepEquals, []string{"c", "a"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{-1, 0, -1, -1}), DeepEquals, []string{"b"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{1, 2, 3, 0}), DeepEquals, []string{"_tidb_rowid", "a", "b", "c"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{1, 0, 2, 3}), DeepEquals, []string{"b", "a", "c", "_tidb_rowid"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{-1, 0, 2, 1}), DeepEquals, []string{"b", "_tidb_rowid", "c"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{2, -1, 0, 1}), DeepEquals, []string{"c", "_tidb_rowid", "a"}) - c.Assert(getColumnNames(s.tableInfo.Core, []int{-1, 1, -1, 0}), DeepEquals, []string{"_tidb_rowid", "b"}) -} - -func (s *tableRestoreSuite) TestInitializeColumns(c *C) { - ccp := &checkpoints.ChunkCheckpoint{} - - defer func() { - s.tr.ignoreColumns = nil - }() - - cases := []struct { - columns []string - ignoreColumns map[string]struct{} - expectedPermutation []int - errPat string - }{ - { - nil, - nil, - []int{0, 1, 2, -1}, - "", - }, - { - nil, - map[string]struct{}{"b": {}}, - []int{0, -1, 2, -1}, - "", - }, - { - []string{"b", "c", "a"}, - nil, - []int{2, 0, 1, -1}, - "", - }, - { - []string{"b", "c", "a"}, - map[string]struct{}{"b": {}}, - []int{2, -1, 1, -1}, - "", - }, - { - []string{"b"}, - nil, - []int{-1, 0, -1, -1}, - "", - }, - { - []string{"_tidb_rowid", "b", "a", "c"}, - nil, - []int{2, 1, 3, 0}, - "", - }, - { - []string{"_tidb_rowid", "b", "a", "c"}, - map[string]struct{}{"b": {}, "_tidb_rowid": {}}, - []int{2, -1, 3, -1}, - "", - }, - { - []string{"_tidb_rowid", "b", "a", "c", "d"}, - nil, - nil, - `unknown columns in header \[d\]`, - }, - { - []string{"e", "b", "c", "d"}, - nil, - nil, - `unknown columns in header \[e d\]`, - }, - } - - for _, testCase := range cases { - ccp.ColumnPermutation = nil - s.tr.ignoreColumns = testCase.ignoreColumns - err := s.tr.initializeColumns(testCase.columns, ccp) - if len(testCase.errPat) > 0 { - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, testCase.errPat) - } else { - c.Assert(ccp.ColumnPermutation, DeepEquals, testCase.expectedPermutation) - } - } -} - -func (s *tableRestoreSuite) TestInitializeColumnsGenerated(c *C) { - p := parser.New() - p.SetSQLMode(mysql.ModeANSIQuotes) - se := tmock.NewContext() - - cases := []struct { - schema string - columns []string - expectedPermutation []int - }{ - { - "CREATE TABLE `table` (a INT, b INT, C INT, d INT AS (a * 2))", - []string{"b", "c", "a"}, - []int{2, 0, 1, -1, -1}, - }, - // all generated columns and none input columns - { - "CREATE TABLE `table` (a bigint as (1 + 2) stored, b text as (sha1(repeat('x', a))) stored)", - []string{}, - []int{-1, -1, -1}, - }, - } - - for _, testCase := range cases { - node, err := p.ParseOneStmt(testCase.schema, "", "") - c.Assert(err, IsNil) - core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) - c.Assert(err, IsNil) - core.State = model.StatePublic - tableInfo := &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} - s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) - c.Assert(err, IsNil) - ccp := &checkpoints.ChunkCheckpoint{} - - err = s.tr.initializeColumns(testCase.columns, ccp) - c.Assert(err, IsNil) - c.Assert(ccp.ColumnPermutation, DeepEquals, testCase.expectedPermutation) - } -} - -func (s *tableRestoreSuite) TestCompareChecksumSuccess(c *C) { - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - - mock.ExpectQuery("SELECT.*tikv_gc_life_time.*"). - WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) - mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). - WithArgs("100h0m0s"). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectQuery("ADMIN CHECKSUM.*"). - WillReturnRows( - sqlmock.NewRows([]string{"Db_name", "Table_name", "Checksum_crc64_xor", "Total_kvs", "Total_bytes"}). - AddRow("db", "table", 1234567890, 12345, 1234567), - ) - mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). - WithArgs("10m"). - WillReturnResult(sqlmock.NewResult(2, 1)) - mock.ExpectClose() - - ctx := MockDoChecksumCtx(db) - remoteChecksum, err := DoChecksum(ctx, s.tr.tableInfo) - c.Assert(err, IsNil) - err = s.tr.compareChecksum(remoteChecksum, verification.MakeKVChecksum(1234567, 12345, 1234567890)) - c.Assert(err, IsNil) - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) -} - -func (s *tableRestoreSuite) TestCompareChecksumFailure(c *C) { - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - - mock.ExpectQuery("SELECT.*tikv_gc_life_time.*"). - WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) - mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). - WithArgs("100h0m0s"). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectQuery("ADMIN CHECKSUM TABLE `db`\\.`table`"). - WillReturnRows( - sqlmock.NewRows([]string{"Db_name", "Table_name", "Checksum_crc64_xor", "Total_kvs", "Total_bytes"}). - AddRow("db", "table", 1234567890, 12345, 1234567), - ) - mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). - WithArgs("10m"). - WillReturnResult(sqlmock.NewResult(2, 1)) - mock.ExpectClose() - - ctx := MockDoChecksumCtx(db) - remoteChecksum, err := DoChecksum(ctx, s.tr.tableInfo) - c.Assert(err, IsNil) - err = s.tr.compareChecksum(remoteChecksum, verification.MakeKVChecksum(9876543, 54321, 1357924680)) - c.Assert(err, ErrorMatches, "checksum mismatched.*") - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) -} - -func (s *tableRestoreSuite) TestAnalyzeTable(c *C) { - db, mock, err := sqlmock.New() - c.Assert(err, IsNil) - - mock.ExpectExec("ANALYZE TABLE `db`\\.`table`"). - WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectClose() - - ctx := context.Background() - defaultSQLMode, err := mysql.GetSQLMode(mysql.DefaultSQLMode) - c.Assert(err, IsNil) - g := glue.NewExternalTiDBGlue(db, defaultSQLMode) - err = s.tr.analyzeTable(ctx, g) - c.Assert(err, IsNil) - - c.Assert(db.Close(), IsNil) - c.Assert(mock.ExpectationsWereMet(), IsNil) -} - -func (s *tableRestoreSuite) TestImportKVSuccess(c *C) { - controller := gomock.NewController(c) - defer controller.Finish() - mockBackend := mock.NewMockBackend(controller) - importer := backend.MakeBackend(mockBackend) - chptCh := make(chan saveCp) - defer close(chptCh) - rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} - go func() { - for scp := range chptCh { - if scp.waitCh != nil { - scp.waitCh <- nil - } - } - }() - - ctx := context.Background() - engineUUID := uuid.New() - - mockBackend.EXPECT(). - CloseEngine(ctx, nil, engineUUID). - Return(nil) - mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID, gomock.Any()). - Return(nil) - mockBackend.EXPECT(). - CleanupEngine(ctx, engineUUID). - Return(nil) - - closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) - c.Assert(err, IsNil) - err = s.tr.importKV(ctx, closedEngine, rc, 1) - c.Assert(err, IsNil) -} - -func (s *tableRestoreSuite) TestImportKVFailure(c *C) { - controller := gomock.NewController(c) - defer controller.Finish() - mockBackend := mock.NewMockBackend(controller) - importer := backend.MakeBackend(mockBackend) - chptCh := make(chan saveCp) - defer close(chptCh) - rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} - go func() { - for scp := range chptCh { - if scp.waitCh != nil { - scp.waitCh <- nil - } - } - }() - - ctx := context.Background() - engineUUID := uuid.New() - - mockBackend.EXPECT(). - CloseEngine(ctx, nil, engineUUID). - Return(nil) - mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID, gomock.Any()). - Return(errors.Annotate(context.Canceled, "fake import error")) - - closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) - c.Assert(err, IsNil) - err = s.tr.importKV(ctx, closedEngine, rc, 1) - c.Assert(err, ErrorMatches, "fake import error.*") -} - -func (s *tableRestoreSuite) TestTableRestoreMetrics(c *C) { - controller := gomock.NewController(c) - defer controller.Finish() - - chunkPendingBase := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) - chunkFinishedBase := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) - engineFinishedBase := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues("imported", metric.TableResultSuccess)) - tableFinishedBase := metric.ReadCounter(metric.TableCounter.WithLabelValues("index_imported", metric.TableResultSuccess)) - - ctx := context.Background() - chptCh := make(chan saveCp) - defer close(chptCh) - cfg := config.NewConfig() - cfg.Mydumper.BatchSize = 1 - cfg.PostRestore.Checksum = config.OpLevelOff - - cfg.Checkpoint.Enable = false - cfg.TiDB.Host = "127.0.0.1" - cfg.TiDB.StatusPort = 10080 - cfg.TiDB.Port = 4000 - cfg.TiDB.PdAddr = "127.0.0.1:2379" - - cfg.Mydumper.SourceDir = "." - cfg.Mydumper.CSV.Header = false - cfg.TikvImporter.Backend = config.BackendImporter - tls, err := cfg.ToTLS() - c.Assert(err, IsNil) - - err = cfg.Adjust(ctx) - c.Assert(err, IsNil) - - cpDB := checkpoints.NewNullCheckpointsDB() - g := mock.NewMockGlue(controller) - rc := &Controller{ - cfg: cfg, - dbMetas: []*mydump.MDDatabaseMeta{ - { - Name: s.tableInfo.DB, - Tables: []*mydump.MDTableMeta{s.tableMeta}, - }, - }, - dbInfos: map[string]*checkpoints.TidbDBInfo{ - s.tableInfo.DB: s.dbInfo, - }, - tableWorkers: worker.NewPool(ctx, 6, "table"), - ioWorkers: worker.NewPool(ctx, 5, "io"), - indexWorkers: worker.NewPool(ctx, 2, "index"), - regionWorkers: worker.NewPool(ctx, 10, "region"), - checksumWorks: worker.NewPool(ctx, 2, "region"), - saveCpCh: chptCh, - pauser: DeliverPauser, - backend: noop.NewNoopBackend(), - tidbGlue: g, - errorSummaries: makeErrorSummaries(log.L()), - tls: tls, - checkpointsDB: cpDB, - closedEngineLimit: worker.NewPool(ctx, 1, "closed_engine"), - store: s.store, - metaMgrBuilder: noopMetaMgrBuilder{}, - diskQuotaLock: newDiskQuotaLock(), - errorMgr: errormanager.New(nil, cfg), - } - go func() { - for scp := range chptCh { - if scp.waitCh != nil { - scp.waitCh <- nil - } - } - }() - db, sqlMock, err := sqlmock.New() - c.Assert(err, IsNil) - g.EXPECT().GetDB().Return(db, nil).AnyTimes() - sqlMock.ExpectQuery("SELECT tidb_version\\(\\);").WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}). - AddRow("Release Version: v5.2.1\nEdition: Community\n")) - - web.BroadcastInitProgress(rc.dbMetas) - - err = rc.restoreTables(ctx) - c.Assert(err, IsNil) - - chunkPending := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) - chunkFinished := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) - c.Assert(chunkPending-chunkPendingBase, Equals, float64(7)) - c.Assert(chunkFinished-chunkFinishedBase, Equals, chunkPending-chunkPendingBase) - - engineFinished := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues("imported", metric.TableResultSuccess)) - c.Assert(engineFinished-engineFinishedBase, Equals, float64(8)) - - tableFinished := metric.ReadCounter(metric.TableCounter.WithLabelValues("index_imported", metric.TableResultSuccess)) - c.Assert(tableFinished-tableFinishedBase, Equals, float64(1)) -} - -func (s *tableRestoreSuite) TestSaveStatusCheckpoint(c *C) { - _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/SlowDownCheckpointUpdate", "sleep(100)") - defer func() { - _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/SlowDownCheckpointUpdate") - }() - - web.BroadcastInitProgress([]*mydump.MDDatabaseMeta{{ - Name: "test", - Tables: []*mydump.MDTableMeta{{DB: "test", Name: "tbl"}}, - }}) - web.BroadcastTableCheckpoint(common.UniqueTable("test", "tbl"), &checkpoints.TableCheckpoint{}) - - saveCpCh := make(chan saveCp) - - rc := &Controller{ - saveCpCh: saveCpCh, - checkpointsDB: checkpoints.NewNullCheckpointsDB(), - } - rc.checkpointsWg.Add(1) - go rc.listenCheckpointUpdates() - - start := time.Now() - err := rc.saveStatusCheckpoint(context.Background(), common.UniqueTable("test", "tbl"), indexEngineID, nil, checkpoints.CheckpointStatusImported) - c.Assert(err, IsNil) - elapsed := time.Since(start) - c.Assert(elapsed, GreaterEqual, time.Millisecond*100) - - close(saveCpCh) - rc.checkpointsWg.Wait() -} - -var _ = Suite(&chunkRestoreSuite{}) - -type chunkRestoreSuite struct { - tableRestoreSuiteBase - cr *chunkRestore -} - -func (s *chunkRestoreSuite) SetUpTest(c *C) { - s.tableRestoreSuiteBase.SetUpTest(c) - - ctx := context.Background() - w := worker.NewPool(ctx, 5, "io") - - chunk := checkpoints.ChunkCheckpoint{ - Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, - FileMeta: s.tr.tableMeta.DataFiles[1].FileMeta, - Chunk: mydump.Chunk{ - Offset: 0, - EndOffset: 37, - PrevRowIDMax: 18, - RowIDMax: 36, - }, - } - - var err error - s.cr, err = newChunkRestore(context.Background(), 1, s.cfg, &chunk, w, s.store, nil) - c.Assert(err, IsNil) -} - -func (s *chunkRestoreSuite) TearDownTest(c *C) { - s.cr.close() -} - -func (s *chunkRestoreSuite) TestDeliverLoopCancel(c *C) { - rc := &Controller{backend: importer.NewMockImporter(nil, "")} - - ctx, cancel := context.WithCancel(context.Background()) - kvsCh := make(chan []deliveredKVs) - go cancel() - _, err := s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, nil, nil, rc) - c.Assert(errors.Cause(err), Equals, context.Canceled) -} - -func (s *chunkRestoreSuite) TestDeliverLoopEmptyData(c *C) { - ctx := context.Background() - - // Open two mock engines. - - controller := gomock.NewController(c) - defer controller.Finish() - mockBackend := mock.NewMockBackend(controller) - importer := backend.MakeBackend(mockBackend) - - mockBackend.EXPECT().OpenEngine(ctx, gomock.Any(), gomock.Any()).Return(nil).Times(2) - mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).AnyTimes() - mockWriter := mock.NewMockEngineWriter(controller) - mockBackend.EXPECT().LocalWriter(ctx, gomock.Any(), gomock.Any()).Return(mockWriter, nil).AnyTimes() - mockWriter.EXPECT(). - AppendRows(ctx, gomock.Any(), gomock.Any(), gomock.Any()). - Return(nil).AnyTimes() - - dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) - c.Assert(err, IsNil) - dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) - c.Assert(err, IsNil) - indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - - // Deliver nothing. - - cfg := &config.Config{} - rc := &Controller{cfg: cfg, backend: importer, diskQuotaLock: newDiskQuotaLock()} - - kvsCh := make(chan []deliveredKVs, 1) - kvsCh <- []deliveredKVs{} - _, err = s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, dataWriter, indexWriter, rc) - c.Assert(err, IsNil) -} - -func (s *chunkRestoreSuite) TestDeliverLoop(c *C) { - ctx := context.Background() - kvsCh := make(chan []deliveredKVs) - mockCols := []string{"c1", "c2"} - - // Open two mock engines. - - controller := gomock.NewController(c) - defer controller.Finish() - mockBackend := mock.NewMockBackend(controller) - importer := backend.MakeBackend(mockBackend) - - mockBackend.EXPECT().OpenEngine(ctx, gomock.Any(), gomock.Any()).Return(nil).Times(2) - // avoid return the same object at each call - mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).Times(1) - mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).Times(1) - mockWriter := mock.NewMockEngineWriter(controller) - mockBackend.EXPECT().LocalWriter(ctx, gomock.Any(), gomock.Any()).Return(mockWriter, nil).AnyTimes() - mockWriter.EXPECT().IsSynced().Return(true).AnyTimes() - - dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) - c.Assert(err, IsNil) - indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) - c.Assert(err, IsNil) - - dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - - // Set up the expected API calls to the data engine... - - mockWriter.EXPECT(). - AppendRows(ctx, s.tr.tableName, mockCols, kv.MakeRowsFromKvPairs([]common.KvPair{ - { - Key: []byte("txxxxxxxx_ryyyyyyyy"), - Val: []byte("value1"), - }, - { - Key: []byte("txxxxxxxx_rwwwwwwww"), - Val: []byte("value2"), - }, - })). - Return(nil) - - // ... and the index engine. - // - // Note: This test assumes data engine is written before the index engine. - - mockWriter.EXPECT(). - AppendRows(ctx, s.tr.tableName, mockCols, kv.MakeRowsFromKvPairs([]common.KvPair{ - { - Key: []byte("txxxxxxxx_izzzzzzzz"), - Val: []byte("index1"), - }, - })). - Return(nil) - - // Now actually start the delivery loop. - - saveCpCh := make(chan saveCp, 2) - go func() { - kvsCh <- []deliveredKVs{ - { - kvs: kv.MakeRowFromKvPairs([]common.KvPair{ - { - Key: []byte("txxxxxxxx_ryyyyyyyy"), - Val: []byte("value1"), - }, - { - Key: []byte("txxxxxxxx_rwwwwwwww"), - Val: []byte("value2"), - }, - { - Key: []byte("txxxxxxxx_izzzzzzzz"), - Val: []byte("index1"), - }, - }), - columns: mockCols, - offset: 12, - rowID: 76, - }, - } - kvsCh <- []deliveredKVs{} - close(kvsCh) - }() - - cfg := &config.Config{} - rc := &Controller{cfg: cfg, saveCpCh: saveCpCh, backend: importer, diskQuotaLock: newDiskQuotaLock()} - - _, err = s.cr.deliverLoop(ctx, kvsCh, s.tr, 0, dataWriter, indexWriter, rc) - c.Assert(err, IsNil) - c.Assert(saveCpCh, HasLen, 2) - c.Assert(s.cr.chunk.Chunk.Offset, Equals, int64(12)) - c.Assert(s.cr.chunk.Chunk.PrevRowIDMax, Equals, int64(76)) - c.Assert(s.cr.chunk.Checksum.SumKVS(), Equals, uint64(3)) -} - -func (s *chunkRestoreSuite) TestEncodeLoop(c *C) { - ctx := context.Background() - kvsCh := make(chan []deliveredKVs, 2) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567895, - }) - c.Assert(err, IsNil) - cfg := config.NewConfig() - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, IsNil) - c.Assert(kvsCh, HasLen, 2) - - kvs := <-kvsCh - c.Assert(kvs, HasLen, 1) - c.Assert(kvs[0].rowID, Equals, int64(19)) - c.Assert(kvs[0].offset, Equals, int64(36)) - c.Assert(kvs[0].columns, DeepEquals, []string(nil)) - - kvs = <-kvsCh - c.Assert(len(kvs), Equals, 0) -} - -func (s *chunkRestoreSuite) TestEncodeLoopCanceled(c *C) { - ctx, cancel := context.WithCancel(context.Background()) - kvsCh := make(chan []deliveredKVs) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567896, - }) - c.Assert(err, IsNil) - - go cancel() - cfg := config.NewConfig() - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(errors.Cause(err), Equals, context.Canceled) - c.Assert(kvsCh, HasLen, 0) -} - -func (s *chunkRestoreSuite) TestEncodeLoopForcedError(c *C) { - ctx := context.Background() - kvsCh := make(chan []deliveredKVs, 2) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567897, - }) - c.Assert(err, IsNil) - - // close the chunk so reading it will result in the "file already closed" error. - s.cr.parser.Close() - - cfg := config.NewConfig() - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, ErrorMatches, `in file .*[/\\]?db\.table\.2\.sql:0 at offset 0:.*file already closed`) - c.Assert(kvsCh, HasLen, 0) -} - -func (s *chunkRestoreSuite) TestEncodeLoopDeliverLimit(c *C) { - ctx := context.Background() - kvsCh := make(chan []deliveredKVs, 4) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567898, - }) - c.Assert(err, IsNil) - - dir := c.MkDir() - fileName := "db.limit.000.csv" - err = os.WriteFile(filepath.Join(dir, fileName), []byte("1,2,3\r\n4,5,6\r\n7,8,9\r"), 0o644) - c.Assert(err, IsNil) - - store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - cfg := config.NewConfig() - - reader, err := store.Open(ctx, fileName) - c.Assert(err, IsNil) - w := worker.NewPool(ctx, 1, "io") - p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, false, nil) - c.Assert(err, IsNil) - s.cr.parser = p - - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - c.Assert(failpoint.Enable( - "github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size", "return(110000000)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/mock-kv-size") - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, IsNil) - - // we have 3 kvs total. after the failpoint injected. - // we will send one kv each time. - count := 0 - for { - kvs, ok := <-kvsCh - if !ok { - break - } - count += 1 - if count <= 3 { - c.Assert(kvs, HasLen, 1) - } - if count == 4 { - // we will send empty kvs before encodeLoop exists - // so, we can receive 4 batch and 1 is empty - c.Assert(kvs, HasLen, 0) - break - } - } -} - -func (s *chunkRestoreSuite) TestEncodeLoopDeliverErrored(c *C) { - ctx := context.Background() - kvsCh := make(chan []deliveredKVs) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := kv.NewTableKVEncoder(s.tr.encTable, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567898, - }) - c.Assert(err, IsNil) - - go func() { - deliverCompleteCh <- deliverResult{ - err: errors.New("fake deliver error"), - } - }() - cfg := config.NewConfig() - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, ErrorMatches, "fake deliver error") - c.Assert(kvsCh, HasLen, 0) -} - -func (s *chunkRestoreSuite) TestEncodeLoopColumnsMismatch(c *C) { - dir := c.MkDir() - fileName := "db.table.000.csv" - err := os.WriteFile(filepath.Join(dir, fileName), []byte("1,2\r\n4,5,6,7\r\n"), 0o644) - c.Assert(err, IsNil) - - store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - - ctx := context.Background() - cfg := config.NewConfig() - errorMgr := errormanager.New(nil, cfg) - rc := &Controller{pauser: DeliverPauser, cfg: cfg, errorMgr: errorMgr} - - reader, err := store.Open(ctx, fileName) - c.Assert(err, IsNil) - w := worker.NewPool(ctx, 5, "io") - p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, false, nil) - c.Assert(err, IsNil) - - err = s.cr.parser.Close() - c.Assert(err, IsNil) - s.cr.parser = p - - kvsCh := make(chan []deliveredKVs, 2) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errorMgr).NewEncoder( - s.tr.encTable, - &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567895, - }) - c.Assert(err, IsNil) - defer kvEncoder.Close() - - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, ErrorMatches, "in file db.table.2.sql:0 at offset 4: column count mismatch, expected 3, got 2") - c.Assert(kvsCh, HasLen, 0) -} - -func (s *chunkRestoreSuite) TestEncodeLoopIgnoreColumnsCSV(c *C) { - cases := []struct { - s string - ignoreColumns []*config.IgnoreColumns - kvs deliveredKVs - header bool - }{ - { - "1,2,3\r\n4,5,6\r\n", - []*config.IgnoreColumns{ - { - DB: "db", - Table: "table", - Columns: []string{"a"}, - }, - }, - deliveredKVs{ - rowID: 1, - offset: 6, - columns: []string{"b", "c"}, - }, - false, - }, - { - "b,c\r\n2,3\r\n5,6\r\n", - []*config.IgnoreColumns{ - { - TableFilter: []string{"db*.tab*"}, - Columns: []string{"b"}, - }, - }, - deliveredKVs{ - rowID: 1, - offset: 9, - columns: []string{"c"}, - }, - true, - }, - } - - for _, cs := range cases { - // reset test - s.SetUpTest(c) - s.testEncodeLoopIgnoreColumnsCSV(c, cs.s, cs.ignoreColumns, cs.kvs, cs.header) - } -} - -func (s *chunkRestoreSuite) testEncodeLoopIgnoreColumnsCSV( - c *C, - f string, - ignoreColumns []*config.IgnoreColumns, - deliverKV deliveredKVs, - header bool, -) { - dir := c.MkDir() - fileName := "db.table.000.csv" - err := os.WriteFile(filepath.Join(dir, fileName), []byte(f), 0o644) - c.Assert(err, IsNil) - - store, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - - ctx := context.Background() - cfg := config.NewConfig() - cfg.Mydumper.IgnoreColumns = ignoreColumns - cfg.Mydumper.CSV.Header = header - rc := &Controller{pauser: DeliverPauser, cfg: cfg} - - reader, err := store.Open(ctx, fileName) - c.Assert(err, IsNil) - w := worker.NewPool(ctx, 5, "io") - p, err := mydump.NewCSVParser(&cfg.Mydumper.CSV, reader, 111, w, cfg.Mydumper.CSV.Header, nil) - c.Assert(err, IsNil) - - err = s.cr.parser.Close() - c.Assert(err, IsNil) - s.cr.parser = p - - kvsCh := make(chan []deliveredKVs, 2) - deliverCompleteCh := make(chan deliverResult) - kvEncoder, err := tidb.NewTiDBBackend(nil, config.ReplaceOnDup, errormanager.New(nil, config.NewConfig())).NewEncoder( - s.tr.encTable, - &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 1234567895, - }) - c.Assert(err, IsNil) - defer kvEncoder.Close() - - _, _, err = s.cr.encodeLoop(ctx, kvsCh, s.tr, s.tr.logger, kvEncoder, deliverCompleteCh, rc) - c.Assert(err, IsNil) - c.Assert(kvsCh, HasLen, 2) - - kvs := <-kvsCh - c.Assert(kvs, HasLen, 2) - c.Assert(kvs[0].rowID, Equals, deliverKV.rowID) - c.Assert(kvs[0].offset, Equals, deliverKV.offset) - c.Assert(kvs[0].columns, DeepEquals, deliverKV.columns) - - kvs = <-kvsCh - c.Assert(len(kvs), Equals, 0) -} - -func (s *chunkRestoreSuite) TestRestore(c *C) { - ctx := context.Background() - - // Open two mock engines - - controller := gomock.NewController(c) - defer controller.Finish() - mockClient := mock.NewMockImportKVClient(controller) - mockDataWriter := mock.NewMockImportKV_WriteEngineClient(controller) - mockIndexWriter := mock.NewMockImportKV_WriteEngineClient(controller) - importer := importer.NewMockImporter(mockClient, "127.0.0.1:2379") - - mockClient.EXPECT().OpenEngine(ctx, gomock.Any()).Return(nil, nil) - mockClient.EXPECT().OpenEngine(ctx, gomock.Any()).Return(nil, nil) - - dataEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, 0) - c.Assert(err, IsNil) - indexEngine, err := importer.OpenEngine(ctx, &backend.EngineConfig{}, s.tr.tableName, -1) - c.Assert(err, IsNil) - dataWriter, err := dataEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - indexWriter, err := indexEngine.LocalWriter(ctx, &backend.LocalWriterConfig{}) - c.Assert(err, IsNil) - - // Expected API sequence - // (we don't care about the actual content, this would be checked in the integrated tests) - - mockClient.EXPECT().WriteEngine(ctx).Return(mockDataWriter, nil) - mockDataWriter.EXPECT().Send(gomock.Any()).Return(nil) - mockDataWriter.EXPECT().Send(gomock.Any()).DoAndReturn(func(req *import_kvpb.WriteEngineRequest) error { - c.Assert(req.GetBatch().GetMutations(), HasLen, 1) - return nil - }) - mockDataWriter.EXPECT().CloseAndRecv().Return(nil, nil) - - mockClient.EXPECT().WriteEngine(ctx).Return(mockIndexWriter, nil) - mockIndexWriter.EXPECT().Send(gomock.Any()).Return(nil) - mockIndexWriter.EXPECT().Send(gomock.Any()).DoAndReturn(func(req *import_kvpb.WriteEngineRequest) error { - c.Assert(req.GetBatch().GetMutations(), HasLen, 1) - return nil - }) - mockIndexWriter.EXPECT().CloseAndRecv().Return(nil, nil) - - // Now actually start the restore loop. - - saveCpCh := make(chan saveCp, 2) - err = s.cr.restore(ctx, s.tr, 0, dataWriter, indexWriter, &Controller{ - cfg: s.cfg, - saveCpCh: saveCpCh, - backend: importer, - pauser: DeliverPauser, - diskQuotaLock: newDiskQuotaLock(), - }) - c.Assert(err, IsNil) - c.Assert(saveCpCh, HasLen, 2) -} - -var _ = Suite(&restoreSchemaSuite{}) - -type restoreSchemaSuite struct { - ctx context.Context - rc *Controller - controller *gomock.Controller - tableInfos []*model.TableInfo -} - -func (s *restoreSchemaSuite) SetUpSuite(c *C) { - ctx := context.Background() - fakeDataDir := c.MkDir() - store, err := storage.NewLocalStorage(fakeDataDir) - c.Assert(err, IsNil) - // restore database schema file - fakeDBName := "fakedb" - // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}-schema-create.sql' - fakeFileName := fmt.Sprintf("%s-schema-create.sql", fakeDBName) - err = store.WriteFile(ctx, fakeFileName, []byte(fmt.Sprintf("CREATE DATABASE %s;", fakeDBName))) - c.Assert(err, IsNil) - // restore table schema files - fakeTableFilesCount := 8 - - p := parser.New() - p.SetSQLMode(mysql.ModeANSIQuotes) - se := tmock.NewContext() - - tableInfos := make([]*model.TableInfo, 0, fakeTableFilesCount) - for i := 1; i <= fakeTableFilesCount; i++ { - fakeTableName := fmt.Sprintf("tbl%d", i) - // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}.{table}-schema.sql' - fakeFileName := fmt.Sprintf("%s.%s-schema.sql", fakeDBName, fakeTableName) - fakeFileContent := fmt.Sprintf("CREATE TABLE %s(i TINYINT);", fakeTableName) - err = store.WriteFile(ctx, fakeFileName, []byte(fakeFileContent)) - c.Assert(err, IsNil) - - node, err := p.ParseOneStmt(fakeFileContent, "", "") - c.Assert(err, IsNil) - core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) - c.Assert(err, IsNil) - core.State = model.StatePublic - tableInfos = append(tableInfos, core) - } - s.tableInfos = tableInfos - // restore view schema files - fakeViewFilesCount := 8 - for i := 1; i <= fakeViewFilesCount; i++ { - fakeViewName := fmt.Sprintf("tbl%d", i) - // please follow the `mydump.defaultFileRouteRules`, matches files like '{schema}.{table}-schema-view.sql' - fakeFileName := fmt.Sprintf("%s.%s-schema-view.sql", fakeDBName, fakeViewName) - fakeFileContent := []byte(fmt.Sprintf("CREATE ALGORITHM=UNDEFINED VIEW `%s` (`i`) AS SELECT `i` FROM `%s`.`%s`;", fakeViewName, fakeDBName, fmt.Sprintf("tbl%d", i))) - err = store.WriteFile(ctx, fakeFileName, fakeFileContent) - c.Assert(err, IsNil) - } - config := config.NewConfig() - config.Mydumper.DefaultFileRules = true - config.Mydumper.CharacterSet = "utf8mb4" - config.App.RegionConcurrency = 8 - mydumpLoader, err := mydump.NewMyDumpLoaderWithStore(ctx, config, store) - c.Assert(err, IsNil) - s.rc = &Controller{ - checkTemplate: NewSimpleTemplate(), - cfg: config, - store: store, - dbMetas: mydumpLoader.GetDatabases(), - checkpointsDB: &checkpoints.NullCheckpointsDB{}, - } -} - -//nolint:interfacer // change test case signature might cause Check failed to find this test case? -func (s *restoreSchemaSuite) SetUpTest(c *C) { - s.controller, s.ctx = gomock.WithContext(context.Background(), c) - mockBackend := mock.NewMockBackend(s.controller) - mockBackend.EXPECT(). - FetchRemoteTableModels(gomock.Any(), gomock.Any()). - AnyTimes(). - Return(s.tableInfos, nil) - mockBackend.EXPECT().Close() - s.rc.backend = backend.MakeBackend(mockBackend) - - mockDB, sqlMock, err := sqlmock.New() - c.Assert(err, IsNil) - for i := 0; i < 17; i++ { - sqlMock.ExpectExec(".*").WillReturnResult(sqlmock.NewResult(int64(i), 1)) - } - mockTiDBGlue := mock.NewMockGlue(s.controller) - mockTiDBGlue.EXPECT().GetDB().AnyTimes().Return(mockDB, nil) - mockTiDBGlue.EXPECT(). - OwnsSQLExecutor(). - AnyTimes(). - Return(true) - parser := parser.New() - mockTiDBGlue.EXPECT(). - GetParser(). - AnyTimes(). - Return(parser) - s.rc.tidbGlue = mockTiDBGlue -} - -func (s *restoreSchemaSuite) TearDownTest(c *C) { - s.rc.Close() - s.controller.Finish() -} - -func (s *restoreSchemaSuite) TestRestoreSchemaSuccessful(c *C) { - // before restore, if sysVars is initialized by other test, the time_zone should be default value - if len(s.rc.sysVars) > 0 { - tz, ok := s.rc.sysVars["time_zone"] - c.Assert(ok, IsTrue) - c.Assert(tz, Equals, "SYSTEM") - } - - s.rc.cfg.TiDB.Vars = map[string]string{ - "time_zone": "UTC", - } - err := s.rc.restoreSchema(s.ctx) - c.Assert(err, IsNil) - - // test after restore schema, sysVars has been updated - tz, ok := s.rc.sysVars["time_zone"] - c.Assert(ok, IsTrue) - c.Assert(tz, Equals, "UTC") -} - -func (s *restoreSchemaSuite) TestRestoreSchemaFailed(c *C) { - injectErr := errors.New("Something wrong") - mockSession := mock.NewMockSession(s.controller) - mockSession.EXPECT(). - Close(). - AnyTimes(). - Return() - mockSession.EXPECT(). - Execute(gomock.Any(), gomock.Any()). - AnyTimes(). - Return(nil, injectErr) - mockTiDBGlue := mock.NewMockGlue(s.controller) - mockTiDBGlue.EXPECT(). - GetSession(gomock.Any()). - AnyTimes(). - Return(mockSession, nil) - s.rc.tidbGlue = mockTiDBGlue - err := s.rc.restoreSchema(s.ctx) - c.Assert(err, NotNil) - c.Assert(errors.ErrorEqual(err, injectErr), IsTrue) -} - -// When restoring a CSV with `-no-schema` and the target table doesn't exist -// then we can't restore the schema as the `Path` is empty. This is to make -// sure this results in the correct error. -// https://github.com/pingcap/br/issues/1394 -func (s *restoreSchemaSuite) TestNoSchemaPath(c *C) { - fakeTable := mydump.MDTableMeta{ - DB: "fakedb", - Name: "fake1", - SchemaFile: mydump.FileInfo{ - TableName: filter.Table{ - Schema: "fakedb", - Name: "fake1", - }, - FileMeta: mydump.SourceFileMeta{ - Path: "", - }, - }, - DataFiles: []mydump.FileInfo{}, - TotalSize: 0, - } - s.rc.dbMetas[0].Tables = append(s.rc.dbMetas[0].Tables, &fakeTable) - err := s.rc.restoreSchema(s.ctx) - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, `table .* schema not found`) - s.rc.dbMetas[0].Tables = s.rc.dbMetas[0].Tables[:len(s.rc.dbMetas[0].Tables)-1] -} - -func (s *restoreSchemaSuite) TestRestoreSchemaContextCancel(c *C) { - childCtx, cancel := context.WithCancel(s.ctx) - mockSession := mock.NewMockSession(s.controller) - mockSession.EXPECT(). - Close(). - AnyTimes(). - Return() - mockSession.EXPECT(). - Execute(gomock.Any(), gomock.Any()). - AnyTimes(). - Do(func(context.Context, string) { cancel() }). - Return(nil, nil) - mockTiDBGlue := mock.NewMockGlue(s.controller) - mockTiDBGlue.EXPECT(). - GetSession(gomock.Any()). - AnyTimes(). - Return(mockSession, nil) - s.rc.tidbGlue = mockTiDBGlue - err := s.rc.restoreSchema(childCtx) - cancel() - c.Assert(err, NotNil) - c.Assert(err, Equals, childCtx.Err()) -} - -func (s *tableRestoreSuite) TestCheckClusterResource(c *C) { - cases := []struct { - mockStoreResponse []byte - mockReplicaResponse []byte - expectMsg string - expectResult bool - expectErrorCount int - }{ - { - []byte(`{ - "count": 1, - "stores": [ - { - "store": { - "id": 2 - }, - "status": { - "available": "24" - } - } - ] - }`), - []byte(`{ - "max-replicas": 1 - }`), - "(.*)Cluster available is rich(.*)", - true, - 0, - }, - { - []byte(`{ - "count": 1, - "stores": [ - { - "store": { - "id": 2 - }, - "status": { - "available": "15" - } - } - ] - }`), - []byte(`{ - "max-replicas": 1 - }`), - "(.*)Cluster doesn't have enough space(.*)", - false, - 1, - }, - } - - ctx := context.Background() - dir := c.MkDir() - file := filepath.Join(dir, "tmp") - f, err := os.Create(file) - c.Assert(err, IsNil) - buf := make([]byte, 16) - // write 16 bytes file into local storage - for i := range buf { - buf[i] = byte('A' + i) - } - _, err = f.Write(buf) - c.Assert(err, IsNil) - mockStore, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - for _, ca := range cases { - server := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { - var err error - if strings.HasSuffix(req.URL.Path, "stores") { - _, err = w.Write(ca.mockStoreResponse) - } else { - _, err = w.Write(ca.mockReplicaResponse) - } - c.Assert(err, IsNil) - })) - - tls := common.NewTLSFromMockServer(server) - template := NewSimpleTemplate() - - url := strings.TrimPrefix(server.URL, "https://") - cfg := &config.Config{TiDB: config.DBStore{PdAddr: url}} - rc := &Controller{cfg: cfg, tls: tls, store: mockStore, checkTemplate: template} - var sourceSize int64 - err = rc.store.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { - sourceSize += size - return nil - }) - c.Assert(err, IsNil) - err = rc.clusterResource(ctx, sourceSize) - c.Assert(err, IsNil) - - c.Assert(template.FailedCount(Critical), Equals, ca.expectErrorCount) - c.Assert(template.Success(), Equals, ca.expectResult) - c.Assert(strings.ReplaceAll(template.Output(), "\n", ""), Matches, ca.expectMsg) - - server.Close() - } -} - -type mockTaskMetaMgr struct { - taskMetaMgr -} - -func (mockTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(tasks []taskMeta) ([]taskMeta, error)) error { - _, err := action([]taskMeta{{ - taskID: 1, - pdCfgs: "", - status: taskMetaStatusInitial, - state: taskStateNormal, - }}) - return err -} - -func (s *tableRestoreSuite) TestCheckClusterRegion(c *C) { - type testCase struct { - stores api.StoresInfo - emptyRegions api.RegionsInfo - expectMsgs []string - expectResult bool - expectErrorCnt int - } - - makeRegions := func(regionCnt int, storeID uint64) []api.RegionInfo { - var regions []api.RegionInfo - for i := 0; i < regionCnt; i++ { - regions = append(regions, api.RegionInfo{Peers: []api.MetaPeer{{Peer: &metapb.Peer{StoreId: storeID}}}}) - } - return regions - } - - testCases := []testCase{ - { - stores: api.StoresInfo{Stores: []*api.StoreInfo{ - {Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 200}}, - }}, - emptyRegions: api.RegionsInfo{ - Regions: append([]api.RegionInfo(nil), makeRegions(100, 1)...), - }, - expectMsgs: []string{".*Cluster doesn't have too many empty regions.*", ".*Cluster region distribution is balanced.*"}, - expectResult: true, - expectErrorCnt: 0, - }, - { - stores: api.StoresInfo{Stores: []*api.StoreInfo{ - {Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 2000}}, - {Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 3100}}, - {Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}}, - }}, - emptyRegions: api.RegionsInfo{ - Regions: append(append(append([]api.RegionInfo(nil), - makeRegions(600, 1)...), - makeRegions(300, 2)...), - makeRegions(1200, 3)...), - }, - expectMsgs: []string{ - ".*TiKV stores \\(3\\) contains more than 1000 empty regions respectively.*", - ".*TiKV stores \\(1\\) contains more than 500 empty regions respectively.*", - ".*Region distribution is unbalanced.*but we expect it should not be less than 0.75.*", - }, - expectResult: false, - expectErrorCnt: 1, - }, - { - stores: api.StoresInfo{Stores: []*api.StoreInfo{ - {Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 1200}}, - {Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 3000}}, - {Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}}, - }}, - expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"}, - expectResult: false, - expectErrorCnt: 1, - }, - { - stores: api.StoresInfo{Stores: []*api.StoreInfo{ - {Store: &api.MetaStore{StoreID: 1}, Status: &api.StoreStatus{RegionCount: 0}}, - {Store: &api.MetaStore{StoreID: 2}, Status: &api.StoreStatus{RegionCount: 2800}}, - {Store: &api.MetaStore{StoreID: 3}, Status: &api.StoreStatus{RegionCount: 2500}}, - }}, - expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"}, - expectResult: false, - expectErrorCnt: 1, - }, - } - - mustMarshal := func(v interface{}) []byte { - data, err := json.Marshal(v) - c.Assert(err, IsNil) - return data - } - - for _, ca := range testCases { - server := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { - var err error - if req.URL.Path == pdStores { - _, err = w.Write(mustMarshal(ca.stores)) - } else if req.URL.Path == pdEmptyRegions { - _, err = w.Write(mustMarshal(ca.emptyRegions)) - } else { - w.WriteHeader(http.StatusNotFound) - } - c.Assert(err, IsNil) - })) - - tls := common.NewTLSFromMockServer(server) - template := NewSimpleTemplate() - - url := strings.TrimPrefix(server.URL, "https://") - cfg := &config.Config{TiDB: config.DBStore{PdAddr: url}} - rc := &Controller{cfg: cfg, tls: tls, taskMgr: mockTaskMetaMgr{}, checkTemplate: template} - - err := rc.checkClusterRegion(context.Background()) - c.Assert(err, IsNil) - c.Assert(template.FailedCount(Critical), Equals, ca.expectErrorCnt) - c.Assert(template.Success(), Equals, ca.expectResult) - - for _, expectMsg := range ca.expectMsgs { - c.Assert(strings.ReplaceAll(template.Output(), "\n", ""), Matches, expectMsg) - } - - server.Close() - } -} - -func (s *tableRestoreSuite) TestCheckHasLargeCSV(c *C) { - cases := []struct { - strictFormat bool - expectMsg string - expectResult bool - expectWarnCount int - dbMetas []*mydump.MDDatabaseMeta - }{ - { - true, - "(.*)Skip the csv size check, because config.StrictFormat is true(.*)", - true, - 0, - nil, - }, - { - false, - "(.*)Source csv files size is proper(.*)", - true, - 0, - []*mydump.MDDatabaseMeta{ - { - Tables: []*mydump.MDTableMeta{ - { - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.KiB, - }, - }, - }, - }, - }, - }, - }, - }, - { - false, - "(.*)large csv: /testPath file exists(.*)", - true, - 1, - []*mydump.MDDatabaseMeta{ - { - Tables: []*mydump.MDTableMeta{ - { - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: "/testPath", - }, - }, - }, - }, - }, - }, - }, - }, - } - dir := c.MkDir() - mockStore, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - - for _, ca := range cases { - template := NewSimpleTemplate() - cfg := &config.Config{Mydumper: config.MydumperRuntime{StrictFormat: ca.strictFormat}} - rc := &Controller{cfg: cfg, checkTemplate: template, store: mockStore} - err := rc.HasLargeCSV(ca.dbMetas) - c.Assert(err, IsNil) - c.Assert(template.FailedCount(Warn), Equals, ca.expectWarnCount) - c.Assert(template.Success(), Equals, ca.expectResult) - c.Assert(strings.ReplaceAll(template.Output(), "\n", ""), Matches, ca.expectMsg) - } -} -func (s *tableRestoreSuite) TestEstimate(c *C) { - ctx := context.Background() - controller := gomock.NewController(c) - defer controller.Finish() - mockBackend := mock.NewMockBackend(controller) - idAlloc := kv.NewPanickingAllocators(0) - tbl, err := tables.TableFromMeta(idAlloc, s.tableInfo.Core) - c.Assert(err, IsNil) - - mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).AnyTimes() - mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()).Return(kv.NewTableKVEncoder(tbl, &kv.SessionOptions{ - SQLMode: s.cfg.TiDB.SQLMode, - Timestamp: 0, - AutoRandomSeed: 0, - })).AnyTimes() - importer := backend.MakeBackend(mockBackend) - s.cfg.TikvImporter.Backend = config.BackendLocal - - template := NewSimpleTemplate() - rc := &Controller{ - cfg: s.cfg, - checkTemplate: template, - store: s.store, - backend: importer, - dbMetas: []*mydump.MDDatabaseMeta{ - { - Name: "db1", - Tables: []*mydump.MDTableMeta{s.tableMeta}, - }, - }, - dbInfos: map[string]*checkpoints.TidbDBInfo{ - "db1": s.dbInfo, - }, - ioWorkers: worker.NewPool(context.Background(), 1, "io"), - } - source, err := rc.estimateSourceData(ctx) - // Because this file is small than region split size so we does not sample it. - c.Assert(err, IsNil) - c.Assert(source, Equals, s.tableMeta.TotalSize) - s.tableMeta.TotalSize = int64(config.SplitRegionSize) - source, err = rc.estimateSourceData(ctx) - c.Assert(err, IsNil) - c.Assert(source, Greater, s.tableMeta.TotalSize) - rc.cfg.TikvImporter.Backend = config.BackendTiDB - source, err = rc.estimateSourceData(ctx) - c.Assert(err, IsNil) - c.Assert(source, Equals, s.tableMeta.TotalSize) -} - -func (s *tableRestoreSuite) TestSchemaIsValid(c *C) { - dir := c.MkDir() - ctx := context.Background() - - case1File := "db1.table1.csv" - mockStore, err := storage.NewLocalStorage(dir) - c.Assert(err, IsNil) - err = mockStore.WriteFile(ctx, case1File, []byte(`"a"`)) - c.Assert(err, IsNil) - - case2File := "db1.table2.csv" - err = mockStore.WriteFile(ctx, case2File, []byte("\"colA\",\"colB\"\n\"a\",\"b\"")) - c.Assert(err, IsNil) - - cases := []struct { - ignoreColumns []*config.IgnoreColumns - expectMsg string - // MsgNum == 0 means the check passed. - MsgNum int - hasHeader bool - dbInfos map[string]*checkpoints.TidbDBInfo - tableMeta *mydump.MDTableMeta - }{ - // Case 1: - // csv has one column without header. - // tidb has the two columns but the second column doesn't have the default value. - // we expect the check failed. - { - nil, - "TiDB schema `db1`.`table1` has 2 columns,and data file has 1 columns, but column colb are missing(.*)", - 1, - false, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table1": { - ID: 1, - DB: "db1", - Name: "table1", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colA has the default value - Name: model.NewCIStr("colA"), - DefaultIsExpr: true, - }, - { - // colB doesn't have the default value - Name: model.NewCIStr("colB"), - FieldType: types.FieldType{ - // not null flag - Flag: 1, - }, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table1", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case1File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 2.1: - // csv has two columns(colA, colB) with the header. - // tidb only has one column(colB). - // we expect the check failed. - { - nil, - "TiDB schema `db1`.`table2` doesn't have column cola,(.*)use tables.ignoreColumns to ignore(.*)", - 1, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table2": { - ID: 1, - DB: "db1", - Name: "table2", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colB has the default value - Name: model.NewCIStr("colB"), - DefaultIsExpr: true, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table2", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 2.2: - // csv has two columns(colA, colB) with the header. - // tidb only has one column(colB). - // we ignore colA by set config tables.IgnoreColumns - // we expect the check success. - { - []*config.IgnoreColumns{ - { - DB: "db1", - Table: "table2", - Columns: []string{"cola"}, - }, - }, - "", - 0, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table2": { - ID: 1, - DB: "db1", - Name: "table2", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colB has the default value - Name: model.NewCIStr("colB"), - DefaultIsExpr: true, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table2", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 2.3: - // csv has two columns(colA, colB) with the header. - // tidb has two columns(colB, colC). - // we ignore colA by set config tables.IgnoreColumns - // colC doesn't have the default value. - // we expect the check failed. - { - []*config.IgnoreColumns{ - { - DB: "db1", - Table: "table2", - Columns: []string{"cola"}, - }, - }, - "TiDB schema `db1`.`table2` doesn't have the default value for colc(.*)", - 1, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table2": { - ID: 1, - DB: "db1", - Name: "table2", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colB has the default value - Name: model.NewCIStr("colB"), - DefaultIsExpr: true, - }, - { - // colC doesn't have the default value - Name: model.NewCIStr("colC"), - FieldType: types.FieldType{ - Flag: 1, - }, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table2", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 2.4: - // csv has two columns(colA, colB) with the header. - // tidb has two columns(colB, colC). - // we ignore colB by set config tables.IgnoreColumns - // colB doesn't have the default value. - // we expect the check failed. - { - []*config.IgnoreColumns{ - { - TableFilter: []string{"`db1`.`table2`"}, - Columns: []string{"colb"}, - }, - }, - "TiDB schema `db1`.`table2`'s column colb cannot be ignored(.*)", - 2, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table2": { - ID: 1, - DB: "db1", - Name: "table2", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colB doesn't have the default value - Name: model.NewCIStr("colB"), - FieldType: types.FieldType{ - Flag: 1, - }, - }, - { - // colC has the default value - Name: model.NewCIStr("colC"), - DefaultIsExpr: true, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table2", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 3: - // table3's schema file not found. - // tidb has no table3. - // we expect the check failed. - { - []*config.IgnoreColumns{ - { - TableFilter: []string{"`db1`.`table2`"}, - Columns: []string{"colb"}, - }, - }, - "TiDB schema `db1`.`table3` doesn't exists(.*)", - 1, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "": {}, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table3", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }, - }, - // Case 4: - // table4 has two datafiles for table. we only check the first file. - // we expect the check success. - { - []*config.IgnoreColumns{ - { - DB: "db1", - Table: "table2", - Columns: []string{"cola"}, - }, - }, - "", - 0, - true, - map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "table2": { - ID: 1, - DB: "db1", - Name: "table2", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - // colB has the default value - Name: model.NewCIStr("colB"), - DefaultIsExpr: true, - }, - }, - }, - }, - }, - }, - }, - &mydump.MDTableMeta{ - DB: "db1", - Name: "table2", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - Type: mydump.SourceTypeCSV, - }, - }, - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: case2File, - // This type will make the check failed. - // but it's the second file for table. - // so it's unreachable so this case will success. - Type: mydump.SourceTypeIgnore, - }, - }, - }, - }, - }, - } - - for _, ca := range cases { - template := NewSimpleTemplate() - cfg := &config.Config{ - Mydumper: config.MydumperRuntime{ - ReadBlockSize: config.ReadBlockSize, - CSV: config.CSVConfig{ - Separator: ",", - Delimiter: `"`, - Header: ca.hasHeader, - NotNull: false, - Null: `\N`, - BackslashEscape: true, - TrimLastSep: false, - }, - IgnoreColumns: ca.ignoreColumns, - }, - } - rc := &Controller{ - cfg: cfg, - checkTemplate: template, - store: mockStore, - dbInfos: ca.dbInfos, - ioWorkers: worker.NewPool(context.Background(), 1, "io"), - } - msgs, err := rc.SchemaIsValid(ctx, ca.tableMeta) - c.Assert(err, IsNil) - c.Assert(msgs, HasLen, ca.MsgNum) - if len(msgs) > 0 { - c.Assert(msgs[0], Matches, ca.expectMsg) - } - } -} - -func (s *tableRestoreSuite) TestGBKEncodedSchemaIsValid(c *C) { - cfg := &config.Config{ - Mydumper: config.MydumperRuntime{ - ReadBlockSize: config.ReadBlockSize, - DataCharacterSet: "gb18030", - DataInvalidCharReplace: string(utf8.RuneError), - CSV: config.CSVConfig{ - Separator: ",", - Delimiter: `"`, - Header: true, - NotNull: false, - Null: `\N`, - BackslashEscape: true, - TrimLastSep: false, - }, - IgnoreColumns: nil, - }, - } - charsetConvertor, err := mydump.NewCharsetConvertor(cfg.Mydumper.DataCharacterSet, cfg.Mydumper.DataInvalidCharReplace) - c.Assert(err, IsNil) - mockStore, err := storage.NewLocalStorage(c.MkDir()) - c.Assert(err, IsNil) - csvContent, err := charsetConvertor.Encode(string([]byte("\"colA\",\"colB\"\n\"a\",\"b\""))) - c.Assert(err, IsNil) - ctx := context.Background() - csvFile := "db1.gbk_table.csv" - err = mockStore.WriteFile(ctx, csvFile, []byte(csvContent)) - c.Assert(err, IsNil) - - rc := &Controller{ - cfg: cfg, - checkTemplate: NewSimpleTemplate(), - store: mockStore, - dbInfos: map[string]*checkpoints.TidbDBInfo{ - "db1": { - Name: "db1", - Tables: map[string]*checkpoints.TidbTableInfo{ - "gbk_table": { - ID: 1, - DB: "db1", - Name: "gbk_table", - Core: &model.TableInfo{ - Columns: []*model.ColumnInfo{ - { - Name: model.NewCIStr("colA"), - FieldType: types.FieldType{ - Flag: 1, - }, - }, - { - Name: model.NewCIStr("colB"), - FieldType: types.FieldType{ - Flag: 1, - }, - }, - }, - }, - }, - }, - }, - }, - ioWorkers: worker.NewPool(ctx, 1, "io"), - } - msgs, err := rc.SchemaIsValid(ctx, &mydump.MDTableMeta{ - DB: "db1", - Name: "gbk_table", - DataFiles: []mydump.FileInfo{ - { - FileMeta: mydump.SourceFileMeta{ - FileSize: 1 * units.TiB, - Path: csvFile, - Type: mydump.SourceTypeCSV, - }, - }, - }, - }) - c.Assert(err, IsNil) - c.Assert(msgs, HasLen, 0) -} - -type testChecksumMgr struct { - checksum RemoteChecksum - callCnt int -} - -func (t *testChecksumMgr) Checksum(ctx context.Context, tableInfo *checkpoints.TidbTableInfo) (*RemoteChecksum, error) { - t.callCnt++ - return &t.checksum, nil + require.Equal(t, err.Error(), err1.Error()) + require.NoError(t, mock.ExpectationsWereMet()) } diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 056435d9af7cf..fa50c1179f939 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -202,17 +202,6 @@ func (tr *TableRestore) restoreEngines(pCtx context.Context, rc *Controller, cp tr.logger.Error("fail to restoreEngines because indexengine is nil") return errors.Errorf("table %v index engine checkpoint not found", tr.tableName) } - // If there is an index engine only, it indicates no data needs to restore. - // So we can change status to imported directly and avoid opening engine. - if len(cp.Engines) == 1 { - if err := rc.saveStatusCheckpoint(pCtx, tr.tableName, indexEngineID, nil, checkpoints.CheckpointStatusImported); err != nil { - return errors.Trace(err) - } - if err := rc.saveStatusCheckpoint(pCtx, tr.tableName, checkpoints.WholeTableEngineID, nil, checkpoints.CheckpointStatusIndexImported); err != nil { - return errors.Trace(err) - } - return nil - } ctx, cancel := context.WithCancel(pCtx) defer cancel() @@ -683,10 +672,7 @@ func (tr *TableRestore) postProcess( forcePostProcess bool, metaMgr tableMetaMgr, ) (bool, error) { - // there are no data in this table, no need to do post process - // this is important for tables that are just the dump table of views - // because at this stage, the table was already deleted and replaced by the related view - if !rc.backend.ShouldPostProcess() || len(cp.Engines) == 1 { + if !rc.backend.ShouldPostProcess() { return false, nil } diff --git a/br/pkg/lightning/restore/table_restore_test.go b/br/pkg/lightning/restore/table_restore_test.go new file mode 100644 index 0000000000000..664203144473b --- /dev/null +++ b/br/pkg/lightning/restore/table_restore_test.go @@ -0,0 +1,1849 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package restore + +import ( + "context" + "encoding/json" + "fmt" + "net/http" + "net/http/httptest" + "os" + "path/filepath" + "strconv" + "strings" + "testing" + "time" + "unicode/utf8" + + "github.com/DATA-DOG/go-sqlmock" + "github.com/docker/go-units" + "github.com/golang/mock/gomock" + "github.com/google/uuid" + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/metapb" + filter "github.com/pingcap/tidb-tools/pkg/table-filter" + "github.com/pingcap/tidb/br/pkg/lightning/backend" + "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/backend/noop" + "github.com/pingcap/tidb/br/pkg/lightning/backend/tidb" + "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" + "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/errormanager" + "github.com/pingcap/tidb/br/pkg/lightning/glue" + "github.com/pingcap/tidb/br/pkg/lightning/log" + "github.com/pingcap/tidb/br/pkg/lightning/metric" + "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/lightning/verification" + "github.com/pingcap/tidb/br/pkg/lightning/web" + "github.com/pingcap/tidb/br/pkg/lightning/worker" + "github.com/pingcap/tidb/br/pkg/mock" + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/store/pdtypes" + "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" + tmock "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" + "github.com/stretchr/testify/suite" +) + +type tableRestoreSuiteBase struct { + tr *TableRestore + cfg *config.Config + + tableInfo *checkpoints.TidbTableInfo + dbInfo *checkpoints.TidbDBInfo + tableMeta *mydump.MDTableMeta + + store storage.ExternalStorage +} + +func (s *tableRestoreSuiteBase) setupSuite(t *testing.T) { + web.EnableCurrentProgress() + // Produce a mock table info + + p := parser.New() + p.SetSQLMode(mysql.ModeANSIQuotes) + se := tmock.NewContext() + node, err := p.ParseOneStmt(` + CREATE TABLE "table" ( + a INT, + b INT, + c INT, + KEY (b) + ) +`, "", "") + require.NoError(t, err) + core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) + require.NoError(t, err) + core.State = model.StatePublic + + s.tableInfo = &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} + s.dbInfo = &checkpoints.TidbDBInfo{ + Name: "db", + Tables: map[string]*checkpoints.TidbTableInfo{"table": s.tableInfo}, + } + + // Write some sample SQL dump + + fakeDataDir := t.TempDir() + store, err := storage.NewLocalStorage(fakeDataDir) + require.NoError(t, err) + s.store = store + + fakeDataFilesCount := 6 + fakeDataFilesContent := []byte("INSERT INTO `table` VALUES (1, 2, 3);") + require.Equal(t, 37, len(fakeDataFilesContent)) + fakeDataFiles := make([]mydump.FileInfo, 0, fakeDataFilesCount) + for i := 1; i <= fakeDataFilesCount; i++ { + fakeFileName := fmt.Sprintf("db.table.%d.sql", i) + fakeDataPath := filepath.Join(fakeDataDir, fakeFileName) + err = os.WriteFile(fakeDataPath, fakeDataFilesContent, 0o644) + require.NoError(t, err) + fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ + TableName: filter.Table{Schema: "db", Name: "table"}, + FileMeta: mydump.SourceFileMeta{ + Path: fakeFileName, + Type: mydump.SourceTypeSQL, + SortKey: strconv.Itoa(i), + FileSize: 37, + }, + }) + } + + fakeCsvContent := []byte("1,2,3\r\n4,5,6\r\n") + csvName := "db.table.99.csv" + err = os.WriteFile(filepath.Join(fakeDataDir, csvName), fakeCsvContent, 0o644) + require.NoError(t, err) + fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ + TableName: filter.Table{Schema: "db", Name: "table"}, + FileMeta: mydump.SourceFileMeta{ + Path: csvName, + Type: mydump.SourceTypeCSV, + SortKey: "99", + FileSize: 14, + }, + }) + + s.tableMeta = &mydump.MDTableMeta{ + DB: "db", + Name: "table", + TotalSize: 222, + SchemaFile: mydump.FileInfo{ + TableName: filter.Table{Schema: "db", Name: "table"}, + FileMeta: mydump.SourceFileMeta{ + Path: "db.table-schema.sql", + Type: mydump.SourceTypeTableSchema, + }, + }, + DataFiles: fakeDataFiles, + } +} + +func (s *tableRestoreSuiteBase) setupTest(t *testing.T) { + // Collect into the test TableRestore structure + var err error + s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) + require.NoError(t, err) + + s.cfg = config.NewConfig() + s.cfg.Mydumper.BatchSize = 111 + s.cfg.App.TableConcurrency = 2 +} + +type tableRestoreSuite struct { + suite.Suite + tableRestoreSuiteBase +} + +func TestTableRestoreSuite(t *testing.T) { + suite.Run(t, new(tableRestoreSuite)) +} + +func (s *tableRestoreSuite) SetupSuite() { + s.setupSuite(s.T()) +} + +func (s *tableRestoreSuite) SetupTest() { + s.setupTest(s.T()) +} + +func (s *tableRestoreSuite) TestPopulateChunks() { + _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp", "return(1234567897)") + defer func() { + _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp") + }() + + cp := &checkpoints.TableCheckpoint{ + Engines: make(map[int32]*checkpoints.EngineCheckpoint), + } + + rc := &Controller{cfg: s.cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: s.store} + err := s.tr.populateChunks(context.Background(), rc, cp) + require.NoError(s.T(), err) + //nolint:dupl // false positive. + require.Equal(s.T(), map[int32]*checkpoints.EngineCheckpoint{ + -1: { + Status: checkpoints.CheckpointStatusLoaded, + }, + 0: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[0].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[0].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 0, + RowIDMax: 7, // 37 bytes with 3 columns can store at most 7 rows. + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[1].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 7, + RowIDMax: 14, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[2].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[2].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 14, + RowIDMax: 21, + }, + Timestamp: 1234567897, + }, + }, + }, + 1: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[3].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[3].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 21, + RowIDMax: 28, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[4].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[4].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 28, + RowIDMax: 35, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[5].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[5].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 37, + PrevRowIDMax: 35, + RowIDMax: 42, + }, + Timestamp: 1234567897, + }, + }, + }, + 2: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: s.tr.tableMeta.DataFiles[6].FileMeta.Path, Offset: 0}, + FileMeta: s.tr.tableMeta.DataFiles[6].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 14, + PrevRowIDMax: 42, + RowIDMax: 46, + }, + Timestamp: 1234567897, + }, + }, + }, + }, cp.Engines) + + // set csv header to true, this will cause check columns fail + s.cfg.Mydumper.CSV.Header = true + s.cfg.Mydumper.StrictFormat = true + regionSize := s.cfg.Mydumper.MaxRegionSize + s.cfg.Mydumper.MaxRegionSize = 5 + err = s.tr.populateChunks(context.Background(), rc, cp) + require.Error(s.T(), err) + require.Regexp(s.T(), `.*unknown columns in header \[1 2 3\]`, err.Error()) + s.cfg.Mydumper.MaxRegionSize = regionSize + s.cfg.Mydumper.CSV.Header = false +} + +type errorLocalWriter struct{} + +func (w errorLocalWriter) AppendRows(context.Context, string, []string, kv.Rows) error { + return errors.New("mock write rows failed") +} + +func (w errorLocalWriter) IsSynced() bool { + return true +} + +func (w errorLocalWriter) Close(context.Context) (backend.ChunkFlushStatus, error) { + return nil, nil +} + +func (s *tableRestoreSuite) TestRestoreEngineFailed() { + ctx := context.Background() + ctrl := gomock.NewController(s.T()) + mockBackend := mock.NewMockBackend(ctrl) + rc := &Controller{ + cfg: s.cfg, + pauser: DeliverPauser, + ioWorkers: worker.NewPool(ctx, 1, "io"), + regionWorkers: worker.NewPool(ctx, 10, "region"), + store: s.store, + backend: backend.MakeBackend(mockBackend), + errorSummaries: makeErrorSummaries(log.L()), + saveCpCh: make(chan saveCp, 1), + diskQuotaLock: newDiskQuotaLock(), + } + defer close(rc.saveCpCh) + go func() { + for cp := range rc.saveCpCh { + cp.waitCh <- nil + } + }() + + cp := &checkpoints.TableCheckpoint{ + Engines: make(map[int32]*checkpoints.EngineCheckpoint), + } + err := s.tr.populateChunks(ctx, rc, cp) + require.NoError(s.T(), err) + + tbl, err := tables.TableFromMeta(kv.NewPanickingAllocators(0), s.tableInfo.Core) + require.NoError(s.T(), err) + _, indexUUID := backend.MakeUUID("`db`.`table`", -1) + _, dataUUID := backend.MakeUUID("`db`.`table`", 0) + realBackend := tidb.NewTiDBBackend(nil, "replace", nil) + mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + mockBackend.EXPECT().CloseEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()). + Return(realBackend.NewEncoder(tbl, &kv.SessionOptions{})). + AnyTimes() + mockBackend.EXPECT().MakeEmptyRows().Return(realBackend.MakeEmptyRows()).AnyTimes() + mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), dataUUID).Return(noop.Writer{}, nil) + mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), indexUUID). + Return(nil, errors.New("mock open index local writer failed")) + openedIdxEngine, err := rc.backend.OpenEngine(ctx, nil, "`db`.`table`", -1) + require.NoError(s.T(), err) + + // open the first engine meet error, should directly return the error + _, err = s.tr.restoreEngine(ctx, rc, openedIdxEngine, 0, cp.Engines[0]) + require.Equal(s.T(), "mock open index local writer failed", err.Error()) + + localWriter := func(ctx context.Context, cfg *backend.LocalWriterConfig, engineUUID uuid.UUID) (backend.EngineWriter, error) { + time.Sleep(20 * time.Millisecond) + select { + case <-ctx.Done(): + return nil, errors.New("mock open index local writer failed after ctx.Done") + default: + return noop.Writer{}, nil + } + } + mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + mockBackend.EXPECT().OpenEngine(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), dataUUID).Return(errorLocalWriter{}, nil).AnyTimes() + mockBackend.EXPECT().LocalWriter(gomock.Any(), gomock.Any(), indexUUID). + DoAndReturn(localWriter).AnyTimes() + + openedIdxEngine, err = rc.backend.OpenEngine(ctx, nil, "`db`.`table`", -1) + require.NoError(s.T(), err) + + // open engine failed after write rows failed, should return write rows error + _, err = s.tr.restoreEngine(ctx, rc, openedIdxEngine, 0, cp.Engines[0]) + require.Equal(s.T(), "mock write rows failed", err.Error()) +} + +func (s *tableRestoreSuite) TestPopulateChunksCSVHeader() { + fakeDataDir := s.T().TempDir() + + store, err := storage.NewLocalStorage(fakeDataDir) + require.NoError(s.T(), err) + + fakeDataFiles := make([]mydump.FileInfo, 0) + + fakeCsvContents := []string{ + // small full header + "a,b,c\r\n1,2,3\r\n", + // small partial header + "b,c\r\n2,3\r\n", + // big full header + "a,b,c\r\n90000,80000,700000\r\n1000,2000,3000\r\n11,22,33\r\n3,4,5\r\n", + // big full header unordered + "c,a,b\r\n,1000,2000,3000\r\n11,22,33\r\n1000,2000,404\r\n3,4,5\r\n90000,80000,700000\r\n7999999,89999999,9999999\r\n", + // big partial header + "b,c\r\n2000001,30000001\r\n35231616,462424626\r\n62432,434898934\r\n", + } + total := 0 + for i, str := range fakeCsvContents { + csvName := fmt.Sprintf("db.table.%02d.csv", i) + err := os.WriteFile(filepath.Join(fakeDataDir, csvName), []byte(str), 0o644) + require.NoError(s.T(), err) + fakeDataFiles = append(fakeDataFiles, mydump.FileInfo{ + TableName: filter.Table{Schema: "db", Name: "table"}, + FileMeta: mydump.SourceFileMeta{Path: csvName, Type: mydump.SourceTypeCSV, SortKey: fmt.Sprintf("%02d", i), FileSize: int64(len(str))}, + }) + total += len(str) + } + tableMeta := &mydump.MDTableMeta{ + DB: "db", + Name: "table", + TotalSize: int64(total), + SchemaFile: mydump.FileInfo{TableName: filter.Table{Schema: "db", Name: "table"}, FileMeta: mydump.SourceFileMeta{Path: "db.table-schema.sql", Type: mydump.SourceTypeTableSchema}}, + DataFiles: fakeDataFiles, + } + + _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp", "return(1234567897)") + defer func() { + _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/PopulateChunkTimestamp") + }() + + cp := &checkpoints.TableCheckpoint{ + Engines: make(map[int32]*checkpoints.EngineCheckpoint), + } + + cfg := config.NewConfig() + cfg.Mydumper.BatchSize = 100 + cfg.Mydumper.MaxRegionSize = 40 + + cfg.Mydumper.CSV.Header = true + cfg.Mydumper.StrictFormat = true + rc := &Controller{cfg: cfg, ioWorkers: worker.NewPool(context.Background(), 1, "io"), store: store} + + tr, err := NewTableRestore("`db`.`table`", tableMeta, s.dbInfo, s.tableInfo, &checkpoints.TableCheckpoint{}, nil) + require.NoError(s.T(), err) + require.NoError(s.T(), tr.populateChunks(context.Background(), rc, cp)) + + require.Equal(s.T(), map[int32]*checkpoints.EngineCheckpoint{ + -1: { + Status: checkpoints.CheckpointStatusLoaded, + }, + 0: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[0].FileMeta.Path, Offset: 0}, + FileMeta: tableMeta.DataFiles[0].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 14, + PrevRowIDMax: 0, + RowIDMax: 4, // 37 bytes with 3 columns can store at most 7 rows. + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[1].FileMeta.Path, Offset: 0}, + FileMeta: tableMeta.DataFiles[1].FileMeta, + Chunk: mydump.Chunk{ + Offset: 0, + EndOffset: 10, + PrevRowIDMax: 4, + RowIDMax: 7, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[2].FileMeta.Path, Offset: 6}, + FileMeta: tableMeta.DataFiles[2].FileMeta, + ColumnPermutation: []int{0, 1, 2, -1}, + Chunk: mydump.Chunk{ + Offset: 6, + EndOffset: 52, + PrevRowIDMax: 7, + RowIDMax: 20, + Columns: []string{"a", "b", "c"}, + }, + + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[2].FileMeta.Path, Offset: 52}, + FileMeta: tableMeta.DataFiles[2].FileMeta, + ColumnPermutation: []int{0, 1, 2, -1}, + Chunk: mydump.Chunk{ + Offset: 52, + EndOffset: 60, + PrevRowIDMax: 20, + RowIDMax: 22, + Columns: []string{"a", "b", "c"}, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 6}, + FileMeta: tableMeta.DataFiles[3].FileMeta, + ColumnPermutation: []int{1, 2, 0, -1}, + Chunk: mydump.Chunk{ + Offset: 6, + EndOffset: 48, + PrevRowIDMax: 22, + RowIDMax: 35, + Columns: []string{"c", "a", "b"}, + }, + Timestamp: 1234567897, + }, + }, + }, + 1: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 48}, + FileMeta: tableMeta.DataFiles[3].FileMeta, + ColumnPermutation: []int{1, 2, 0, -1}, + Chunk: mydump.Chunk{ + Offset: 48, + EndOffset: 101, + PrevRowIDMax: 35, + RowIDMax: 48, + Columns: []string{"c", "a", "b"}, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[3].FileMeta.Path, Offset: 101}, + FileMeta: tableMeta.DataFiles[3].FileMeta, + ColumnPermutation: []int{1, 2, 0, -1}, + Chunk: mydump.Chunk{ + Offset: 101, + EndOffset: 102, + PrevRowIDMax: 48, + RowIDMax: 48, + Columns: []string{"c", "a", "b"}, + }, + Timestamp: 1234567897, + }, + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[4].FileMeta.Path, Offset: 4}, + FileMeta: tableMeta.DataFiles[4].FileMeta, + ColumnPermutation: []int{-1, 0, 1, -1}, + Chunk: mydump.Chunk{ + Offset: 4, + EndOffset: 59, + PrevRowIDMax: 48, + RowIDMax: 61, + Columns: []string{"b", "c"}, + }, + Timestamp: 1234567897, + }, + }, + }, + 2: { + Status: checkpoints.CheckpointStatusLoaded, + Chunks: []*checkpoints.ChunkCheckpoint{ + { + Key: checkpoints.ChunkCheckpointKey{Path: tableMeta.DataFiles[4].FileMeta.Path, Offset: 59}, + FileMeta: tableMeta.DataFiles[4].FileMeta, + ColumnPermutation: []int{-1, 0, 1, -1}, + Chunk: mydump.Chunk{ + Offset: 59, + EndOffset: 60, + PrevRowIDMax: 61, + RowIDMax: 61, + Columns: []string{"b", "c"}, + }, + Timestamp: 1234567897, + }, + }, + }, + }, cp.Engines) +} + +func (s *tableRestoreSuite) TestGetColumnsNames() { + require.Equal(s.T(), []string{"a", "b", "c"}, getColumnNames(s.tableInfo.Core, []int{0, 1, 2, -1})) + require.Equal(s.T(), []string{"b", "a", "c"}, getColumnNames(s.tableInfo.Core, []int{1, 0, 2, -1})) + require.Equal(s.T(), []string{"b", "c"}, getColumnNames(s.tableInfo.Core, []int{-1, 0, 1, -1})) + require.Equal(s.T(), []string{"a", "b"}, getColumnNames(s.tableInfo.Core, []int{0, 1, -1, -1})) + require.Equal(s.T(), []string{"c", "a"}, getColumnNames(s.tableInfo.Core, []int{1, -1, 0, -1})) + require.Equal(s.T(), []string{"b"}, getColumnNames(s.tableInfo.Core, []int{-1, 0, -1, -1})) + require.Equal(s.T(), []string{"_tidb_rowid", "a", "b", "c"}, getColumnNames(s.tableInfo.Core, []int{1, 2, 3, 0})) + require.Equal(s.T(), []string{"b", "a", "c", "_tidb_rowid"}, getColumnNames(s.tableInfo.Core, []int{1, 0, 2, 3})) + require.Equal(s.T(), []string{"b", "_tidb_rowid", "c"}, getColumnNames(s.tableInfo.Core, []int{-1, 0, 2, 1})) + require.Equal(s.T(), []string{"c", "_tidb_rowid", "a"}, getColumnNames(s.tableInfo.Core, []int{2, -1, 0, 1})) + require.Equal(s.T(), []string{"_tidb_rowid", "b"}, getColumnNames(s.tableInfo.Core, []int{-1, 1, -1, 0})) +} + +func (s *tableRestoreSuite) TestInitializeColumns() { + ccp := &checkpoints.ChunkCheckpoint{} + + defer func() { + s.tr.ignoreColumns = nil + }() + + cases := []struct { + columns []string + ignoreColumns map[string]struct{} + expectedPermutation []int + errPat string + }{ + { + nil, + nil, + []int{0, 1, 2, -1}, + "", + }, + { + nil, + map[string]struct{}{"b": {}}, + []int{0, -1, 2, -1}, + "", + }, + { + []string{"b", "c", "a"}, + nil, + []int{2, 0, 1, -1}, + "", + }, + { + []string{"b", "c", "a"}, + map[string]struct{}{"b": {}}, + []int{2, -1, 1, -1}, + "", + }, + { + []string{"b"}, + nil, + []int{-1, 0, -1, -1}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c"}, + nil, + []int{2, 1, 3, 0}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c"}, + map[string]struct{}{"b": {}, "_tidb_rowid": {}}, + []int{2, -1, 3, -1}, + "", + }, + { + []string{"_tidb_rowid", "b", "a", "c", "d"}, + nil, + nil, + `unknown columns in header \[d\]`, + }, + { + []string{"e", "b", "c", "d"}, + nil, + nil, + `unknown columns in header \[e d\]`, + }, + } + + for _, testCase := range cases { + ccp.ColumnPermutation = nil + s.tr.ignoreColumns = testCase.ignoreColumns + err := s.tr.initializeColumns(testCase.columns, ccp) + if len(testCase.errPat) > 0 { + require.Error(s.T(), err) + require.Regexp(s.T(), testCase.errPat, err.Error()) + } else { + require.Equal(s.T(), testCase.expectedPermutation, ccp.ColumnPermutation) + } + } +} +func (s *tableRestoreSuite) TestInitializeColumnsGenerated() { + p := parser.New() + p.SetSQLMode(mysql.ModeANSIQuotes) + se := tmock.NewContext() + + cases := []struct { + schema string + columns []string + expectedPermutation []int + }{ + { + "CREATE TABLE `table` (a INT, b INT, C INT, d INT AS (a * 2))", + []string{"b", "c", "a"}, + []int{2, 0, 1, -1, -1}, + }, + // all generated columns and none input columns + { + "CREATE TABLE `table` (a bigint as (1 + 2) stored, b text as (sha1(repeat('x', a))) stored)", + []string{}, + []int{-1, -1, -1}, + }, + } + + for _, testCase := range cases { + node, err := p.ParseOneStmt(testCase.schema, "", "") + require.NoError(s.T(), err) + core, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 0xabcdef) + require.NoError(s.T(), err) + core.State = model.StatePublic + tableInfo := &checkpoints.TidbTableInfo{Name: "table", DB: "db", Core: core} + s.tr, err = NewTableRestore("`db`.`table`", s.tableMeta, s.dbInfo, tableInfo, &checkpoints.TableCheckpoint{}, nil) + require.NoError(s.T(), err) + ccp := &checkpoints.ChunkCheckpoint{} + + err = s.tr.initializeColumns(testCase.columns, ccp) + require.NoError(s.T(), err) + require.Equal(s.T(), testCase.expectedPermutation, ccp.ColumnPermutation) + } +} + +func (s *tableRestoreSuite) TestCompareChecksumSuccess() { + db, mock, err := sqlmock.New() + require.NoError(s.T(), err) + defer func() { + require.NoError(s.T(), db.Close()) + require.NoError(s.T(), mock.ExpectationsWereMet()) + }() + + mock.ExpectQuery("SELECT.*tikv_gc_life_time.*"). + WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) + mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). + WithArgs("100h0m0s"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectQuery("ADMIN CHECKSUM.*"). + WillReturnRows( + sqlmock.NewRows([]string{"Db_name", "Table_name", "Checksum_crc64_xor", "Total_kvs", "Total_bytes"}). + AddRow("db", "table", 1234567890, 12345, 1234567), + ) + mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). + WithArgs("10m"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mock.ExpectClose() + + ctx := MockDoChecksumCtx(db) + remoteChecksum, err := DoChecksum(ctx, s.tr.tableInfo) + require.NoError(s.T(), err) + err = s.tr.compareChecksum(remoteChecksum, verification.MakeKVChecksum(1234567, 12345, 1234567890)) + require.NoError(s.T(), err) +} + +func (s *tableRestoreSuite) TestCompareChecksumFailure() { + db, mock, err := sqlmock.New() + require.NoError(s.T(), err) + defer func() { + require.NoError(s.T(), db.Close()) + require.NoError(s.T(), mock.ExpectationsWereMet()) + }() + + mock.ExpectQuery("SELECT.*tikv_gc_life_time.*"). + WillReturnRows(sqlmock.NewRows([]string{"VARIABLE_VALUE"}).AddRow("10m")) + mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). + WithArgs("100h0m0s"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectQuery("ADMIN CHECKSUM TABLE `db`\\.`table`"). + WillReturnRows( + sqlmock.NewRows([]string{"Db_name", "Table_name", "Checksum_crc64_xor", "Total_kvs", "Total_bytes"}). + AddRow("db", "table", 1234567890, 12345, 1234567), + ) + mock.ExpectExec("UPDATE.*tikv_gc_life_time.*"). + WithArgs("10m"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mock.ExpectClose() + + ctx := MockDoChecksumCtx(db) + remoteChecksum, err := DoChecksum(ctx, s.tr.tableInfo) + require.NoError(s.T(), err) + err = s.tr.compareChecksum(remoteChecksum, verification.MakeKVChecksum(9876543, 54321, 1357924680)) + require.Regexp(s.T(), "checksum mismatched.*", err.Error()) +} + +func (s *tableRestoreSuite) TestAnalyzeTable() { + db, mock, err := sqlmock.New() + require.NoError(s.T(), err) + defer func() { + require.NoError(s.T(), db.Close()) + require.NoError(s.T(), mock.ExpectationsWereMet()) + }() + + mock.ExpectExec("ANALYZE TABLE `db`\\.`table`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectClose() + + ctx := context.Background() + defaultSQLMode, err := mysql.GetSQLMode(mysql.DefaultSQLMode) + require.NoError(s.T(), err) + g := glue.NewExternalTiDBGlue(db, defaultSQLMode) + err = s.tr.analyzeTable(ctx, g) + require.NoError(s.T(), err) +} + +func (s *tableRestoreSuite) TestImportKVSuccess() { + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockBackend := mock.NewMockBackend(controller) + importer := backend.MakeBackend(mockBackend) + chptCh := make(chan saveCp) + defer close(chptCh) + rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} + go func() { + for scp := range chptCh { + if scp.waitCh != nil { + scp.waitCh <- nil + } + } + }() + + ctx := context.Background() + engineUUID := uuid.New() + + mockBackend.EXPECT(). + CloseEngine(ctx, nil, engineUUID). + Return(nil) + mockBackend.EXPECT(). + ImportEngine(ctx, engineUUID, gomock.Any()). + Return(nil) + mockBackend.EXPECT(). + CleanupEngine(ctx, engineUUID). + Return(nil) + + closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) + require.NoError(s.T(), err) + err = s.tr.importKV(ctx, closedEngine, rc, 1) + require.NoError(s.T(), err) +} + +func (s *tableRestoreSuite) TestImportKVFailure() { + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockBackend := mock.NewMockBackend(controller) + importer := backend.MakeBackend(mockBackend) + chptCh := make(chan saveCp) + defer close(chptCh) + rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} + go func() { + for scp := range chptCh { + if scp.waitCh != nil { + scp.waitCh <- nil + } + } + }() + + ctx := context.Background() + engineUUID := uuid.New() + + mockBackend.EXPECT(). + CloseEngine(ctx, nil, engineUUID). + Return(nil) + mockBackend.EXPECT(). + ImportEngine(ctx, engineUUID, gomock.Any()). + Return(errors.Annotate(context.Canceled, "fake import error")) + + closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) + require.NoError(s.T(), err) + err = s.tr.importKV(ctx, closedEngine, rc, 1) + require.Regexp(s.T(), "fake import error.*", err.Error()) +} + +func (s *tableRestoreSuite) TestTableRestoreMetrics() { + controller := gomock.NewController(s.T()) + defer controller.Finish() + + chunkPendingBase := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) + chunkFinishedBase := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) + engineFinishedBase := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues("imported", metric.TableResultSuccess)) + tableFinishedBase := metric.ReadCounter(metric.TableCounter.WithLabelValues("index_imported", metric.TableResultSuccess)) + + ctx := context.Background() + chptCh := make(chan saveCp) + defer close(chptCh) + cfg := config.NewConfig() + cfg.Mydumper.BatchSize = 1 + cfg.PostRestore.Checksum = config.OpLevelOff + + cfg.Checkpoint.Enable = false + cfg.TiDB.Host = "127.0.0.1" + cfg.TiDB.StatusPort = 10080 + cfg.TiDB.Port = 4000 + cfg.TiDB.PdAddr = "127.0.0.1:2379" + + cfg.Mydumper.SourceDir = "." + cfg.Mydumper.CSV.Header = false + cfg.TikvImporter.Backend = config.BackendImporter + tls, err := cfg.ToTLS() + require.NoError(s.T(), err) + + err = cfg.Adjust(ctx) + require.NoError(s.T(), err) + + cpDB := checkpoints.NewNullCheckpointsDB() + g := mock.NewMockGlue(controller) + rc := &Controller{ + cfg: cfg, + dbMetas: []*mydump.MDDatabaseMeta{ + { + Name: s.tableInfo.DB, + Tables: []*mydump.MDTableMeta{s.tableMeta}, + }, + }, + dbInfos: map[string]*checkpoints.TidbDBInfo{ + s.tableInfo.DB: s.dbInfo, + }, + tableWorkers: worker.NewPool(ctx, 6, "table"), + ioWorkers: worker.NewPool(ctx, 5, "io"), + indexWorkers: worker.NewPool(ctx, 2, "index"), + regionWorkers: worker.NewPool(ctx, 10, "region"), + checksumWorks: worker.NewPool(ctx, 2, "region"), + saveCpCh: chptCh, + pauser: DeliverPauser, + backend: noop.NewNoopBackend(), + tidbGlue: g, + errorSummaries: makeErrorSummaries(log.L()), + tls: tls, + checkpointsDB: cpDB, + closedEngineLimit: worker.NewPool(ctx, 1, "closed_engine"), + store: s.store, + metaMgrBuilder: noopMetaMgrBuilder{}, + diskQuotaLock: newDiskQuotaLock(), + errorMgr: errormanager.New(nil, cfg), + } + go func() { + for scp := range chptCh { + if scp.waitCh != nil { + scp.waitCh <- nil + } + } + }() + db, sqlMock, err := sqlmock.New() + require.NoError(s.T(), err) + g.EXPECT().GetDB().Return(db, nil).AnyTimes() + sqlMock.ExpectQuery("SELECT tidb_version\\(\\);").WillReturnRows(sqlmock.NewRows([]string{"tidb_version()"}). + AddRow("Release Version: v5.2.1\nEdition: Community\n")) + + web.BroadcastInitProgress(rc.dbMetas) + + err = rc.restoreTables(ctx) + require.NoError(s.T(), err) + + chunkPending := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) + chunkFinished := metric.ReadCounter(metric.ChunkCounter.WithLabelValues(metric.ChunkStatePending)) + require.Equal(s.T(), float64(7), chunkPending-chunkPendingBase) + require.Equal(s.T(), chunkPending-chunkPendingBase, chunkFinished-chunkFinishedBase) + + engineFinished := metric.ReadCounter(metric.ProcessedEngineCounter.WithLabelValues("imported", metric.TableResultSuccess)) + require.Equal(s.T(), float64(8), engineFinished-engineFinishedBase) + + tableFinished := metric.ReadCounter(metric.TableCounter.WithLabelValues("index_imported", metric.TableResultSuccess)) + require.Equal(s.T(), float64(1), tableFinished-tableFinishedBase) +} + +func (s *tableRestoreSuite) TestSaveStatusCheckpoint() { + _ = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/restore/SlowDownCheckpointUpdate", "sleep(100)") + defer func() { + _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/restore/SlowDownCheckpointUpdate") + }() + + web.BroadcastInitProgress([]*mydump.MDDatabaseMeta{{ + Name: "test", + Tables: []*mydump.MDTableMeta{{DB: "test", Name: "tbl"}}, + }}) + web.BroadcastTableCheckpoint(common.UniqueTable("test", "tbl"), &checkpoints.TableCheckpoint{}) + + saveCpCh := make(chan saveCp) + + rc := &Controller{ + saveCpCh: saveCpCh, + checkpointsDB: checkpoints.NewNullCheckpointsDB(), + } + rc.checkpointsWg.Add(1) + go rc.listenCheckpointUpdates() + + start := time.Now() + err := rc.saveStatusCheckpoint(context.Background(), common.UniqueTable("test", "tbl"), indexEngineID, nil, checkpoints.CheckpointStatusImported) + require.NoError(s.T(), err) + elapsed := time.Since(start) + require.GreaterOrEqual(s.T(), elapsed, time.Millisecond*100) + + close(saveCpCh) + rc.checkpointsWg.Wait() +} + +func (s *tableRestoreSuite) TestCheckClusterResource() { + cases := []struct { + mockStoreResponse []byte + mockReplicaResponse []byte + expectMsg string + expectResult bool + expectErrorCount int + }{ + { + []byte(`{ + "count": 1, + "stores": [ + { + "store": { + "id": 2 + }, + "status": { + "available": "24" + } + } + ] + }`), + []byte(`{ + "max-replicas": 1 + }`), + "(.*)Cluster available is rich(.*)", + true, + 0, + }, + { + []byte(`{ + "count": 1, + "stores": [ + { + "store": { + "id": 2 + }, + "status": { + "available": "15" + } + } + ] + }`), + []byte(`{ + "max-replicas": 1 + }`), + "(.*)Cluster doesn't have enough space(.*)", + false, + 1, + }, + } + + ctx := context.Background() + dir := s.T().TempDir() + + file := filepath.Join(dir, "tmp") + f, err := os.Create(file) + require.NoError(s.T(), err) + buf := make([]byte, 16) + // write 16 bytes file into local storage + for i := range buf { + buf[i] = byte('A' + i) + } + _, err = f.Write(buf) + require.NoError(s.T(), err) + mockStore, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + for _, ca := range cases { + server := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + var err error + if strings.HasSuffix(req.URL.Path, "stores") { + _, err = w.Write(ca.mockStoreResponse) + } else { + _, err = w.Write(ca.mockReplicaResponse) + } + require.NoError(s.T(), err) + })) + + tls := common.NewTLSFromMockServer(server) + template := NewSimpleTemplate() + + url := strings.TrimPrefix(server.URL, "https://") + cfg := &config.Config{TiDB: config.DBStore{PdAddr: url}} + rc := &Controller{cfg: cfg, tls: tls, store: mockStore, checkTemplate: template} + var sourceSize int64 + err = rc.store.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + sourceSize += size + return nil + }) + require.NoError(s.T(), err) + err = rc.clusterResource(ctx, sourceSize) + require.NoError(s.T(), err) + + require.Equal(s.T(), ca.expectErrorCount, template.FailedCount(Critical)) + require.Equal(s.T(), ca.expectResult, template.Success()) + require.Regexp(s.T(), ca.expectMsg, strings.ReplaceAll(template.Output(), "\n", "")) + + server.Close() + } +} + +type mockTaskMetaMgr struct { + taskMetaMgr +} + +func (mockTaskMetaMgr) CheckTasksExclusively(ctx context.Context, action func(tasks []taskMeta) ([]taskMeta, error)) error { + _, err := action([]taskMeta{{ + taskID: 1, + pdCfgs: "", + status: taskMetaStatusInitial, + state: taskStateNormal, + }}) + return err +} + +func (s *tableRestoreSuite) TestCheckClusterRegion() { + type testCase struct { + stores pdtypes.StoresInfo + emptyRegions pdtypes.RegionsInfo + expectMsgs []string + expectResult bool + expectErrorCnt int + } + + makeRegions := func(regionCnt int, storeID uint64) []pdtypes.RegionInfo { + var regions []pdtypes.RegionInfo + for i := 0; i < regionCnt; i++ { + regions = append(regions, pdtypes.RegionInfo{Peers: []pdtypes.MetaPeer{{Peer: &metapb.Peer{StoreId: storeID}}}}) + } + return regions + } + + testCases := []testCase{ + { + stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{ + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 200}}, + }}, + emptyRegions: pdtypes.RegionsInfo{ + Regions: append([]pdtypes.RegionInfo(nil), makeRegions(100, 1)...), + }, + expectMsgs: []string{".*Cluster doesn't have too many empty regions.*", ".*Cluster region distribution is balanced.*"}, + expectResult: true, + expectErrorCnt: 0, + }, + { + stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{ + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 2000}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 3100}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}}, + }}, + emptyRegions: pdtypes.RegionsInfo{ + Regions: append(append(append([]pdtypes.RegionInfo(nil), + makeRegions(600, 1)...), + makeRegions(300, 2)...), + makeRegions(1200, 3)...), + }, + expectMsgs: []string{ + ".*TiKV stores \\(3\\) contains more than 1000 empty regions respectively.*", + ".*TiKV stores \\(1\\) contains more than 500 empty regions respectively.*", + ".*Region distribution is unbalanced.*but we expect it should not be less than 0.75.*", + }, + expectResult: false, + expectErrorCnt: 1, + }, + { + stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{ + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 1200}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 3000}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}}, + }}, + expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"}, + expectResult: false, + expectErrorCnt: 1, + }, + { + stores: pdtypes.StoresInfo{Stores: []*pdtypes.StoreInfo{ + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 1}}, Status: &pdtypes.StoreStatus{RegionCount: 0}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 2}}, Status: &pdtypes.StoreStatus{RegionCount: 2800}}, + {Store: &pdtypes.MetaStore{Store: &metapb.Store{Id: 3}}, Status: &pdtypes.StoreStatus{RegionCount: 2500}}, + }}, + expectMsgs: []string{".*Region distribution is unbalanced.*but we expect it must not be less than 0.5.*"}, + expectResult: false, + expectErrorCnt: 1, + }, + } + + mustMarshal := func(v interface{}) []byte { + data, err := json.Marshal(v) + require.NoError(s.T(), err) + return data + } + + for _, ca := range testCases { + server := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { + var err error + if req.URL.Path == pdStores { + _, err = w.Write(mustMarshal(ca.stores)) + } else if req.URL.Path == pdEmptyRegions { + _, err = w.Write(mustMarshal(ca.emptyRegions)) + } else { + w.WriteHeader(http.StatusNotFound) + } + require.NoError(s.T(), err) + })) + + tls := common.NewTLSFromMockServer(server) + template := NewSimpleTemplate() + + url := strings.TrimPrefix(server.URL, "https://") + cfg := &config.Config{TiDB: config.DBStore{PdAddr: url}} + rc := &Controller{cfg: cfg, tls: tls, taskMgr: mockTaskMetaMgr{}, checkTemplate: template} + + err := rc.checkClusterRegion(context.Background()) + require.NoError(s.T(), err) + require.Equal(s.T(), ca.expectErrorCnt, template.FailedCount(Critical)) + require.Equal(s.T(), ca.expectResult, template.Success()) + + for _, expectMsg := range ca.expectMsgs { + require.Regexp(s.T(), expectMsg, strings.ReplaceAll(template.Output(), "\n", "")) + } + + server.Close() + } +} + +func (s *tableRestoreSuite) TestCheckHasLargeCSV() { + cases := []struct { + strictFormat bool + expectMsg string + expectResult bool + expectWarnCount int + dbMetas []*mydump.MDDatabaseMeta + }{ + { + true, + "(.*)Skip the csv size check, because config.StrictFormat is true(.*)", + true, + 0, + nil, + }, + { + false, + "(.*)Source csv files size is proper(.*)", + true, + 0, + []*mydump.MDDatabaseMeta{ + { + Tables: []*mydump.MDTableMeta{ + { + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.KiB, + }, + }, + }, + }, + }, + }, + }, + }, + { + false, + "(.*)large csv: /testPath file exists(.*)", + true, + 1, + []*mydump.MDDatabaseMeta{ + { + Tables: []*mydump.MDTableMeta{ + { + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: "/testPath", + }, + }, + }, + }, + }, + }, + }, + }, + } + dir := s.T().TempDir() + + mockStore, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + + for _, ca := range cases { + template := NewSimpleTemplate() + cfg := &config.Config{Mydumper: config.MydumperRuntime{StrictFormat: ca.strictFormat}} + rc := &Controller{cfg: cfg, checkTemplate: template, store: mockStore} + err := rc.HasLargeCSV(ca.dbMetas) + require.NoError(s.T(), err) + require.Equal(s.T(), ca.expectWarnCount, template.FailedCount(Warn)) + require.Equal(s.T(), ca.expectResult, template.Success()) + require.Regexp(s.T(), ca.expectMsg, strings.ReplaceAll(template.Output(), "\n", "")) + } +} + +func (s *tableRestoreSuite) TestEstimate() { + ctx := context.Background() + controller := gomock.NewController(s.T()) + defer controller.Finish() + mockBackend := mock.NewMockBackend(controller) + idAlloc := kv.NewPanickingAllocators(0) + tbl, err := tables.TableFromMeta(idAlloc, s.tableInfo.Core) + require.NoError(s.T(), err) + + mockBackend.EXPECT().MakeEmptyRows().Return(kv.MakeRowsFromKvPairs(nil)).AnyTimes() + mockBackend.EXPECT().NewEncoder(gomock.Any(), gomock.Any()).Return(kv.NewTableKVEncoder(tbl, &kv.SessionOptions{ + SQLMode: s.cfg.TiDB.SQLMode, + Timestamp: 0, + AutoRandomSeed: 0, + })).AnyTimes() + importer := backend.MakeBackend(mockBackend) + s.cfg.TikvImporter.Backend = config.BackendLocal + + template := NewSimpleTemplate() + rc := &Controller{ + cfg: s.cfg, + checkTemplate: template, + store: s.store, + backend: importer, + dbMetas: []*mydump.MDDatabaseMeta{ + { + Name: "db1", + Tables: []*mydump.MDTableMeta{s.tableMeta}, + }, + }, + dbInfos: map[string]*checkpoints.TidbDBInfo{ + "db1": s.dbInfo, + }, + ioWorkers: worker.NewPool(context.Background(), 1, "io"), + } + source, err := rc.estimateSourceData(ctx) + // Because this file is small than region split size so we does not sample it. + require.NoError(s.T(), err) + require.Equal(s.T(), s.tableMeta.TotalSize, source) + s.tableMeta.TotalSize = int64(config.SplitRegionSize) + source, err = rc.estimateSourceData(ctx) + require.NoError(s.T(), err) + require.Greater(s.T(), source, s.tableMeta.TotalSize) + rc.cfg.TikvImporter.Backend = config.BackendTiDB + source, err = rc.estimateSourceData(ctx) + require.NoError(s.T(), err) + require.Equal(s.T(), s.tableMeta.TotalSize, source) +} + +func (s *tableRestoreSuite) TestSchemaIsValid() { + dir := s.T().TempDir() + + ctx := context.Background() + + case1File := "db1.table1.csv" + mockStore, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + err = mockStore.WriteFile(ctx, case1File, []byte(`"a"`)) + require.NoError(s.T(), err) + + case2File := "db1.table2.csv" + err = mockStore.WriteFile(ctx, case2File, []byte("\"colA\",\"colB\"\n\"a\",\"b\"")) + require.NoError(s.T(), err) + + cases := []struct { + ignoreColumns []*config.IgnoreColumns + expectMsg string + // MsgNum == 0 means the check passed. + MsgNum int + hasHeader bool + dbInfos map[string]*checkpoints.TidbDBInfo + tableMeta *mydump.MDTableMeta + }{ + // Case 1: + // csv has one column without header. + // tidb has the two columns but the second column doesn't have the default value. + // we expect the check failed. + { + nil, + "TiDB schema `db1`.`table1` has 2 columns,and data file has 1 columns, but column colb are missing(.*)", + 1, + false, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table1": { + ID: 1, + DB: "db1", + Name: "table1", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colA has the default value + Name: model.NewCIStr("colA"), + DefaultIsExpr: true, + }, + { + // colB doesn't have the default value + Name: model.NewCIStr("colB"), + FieldType: types.FieldType{ + // not null flag + Flag: 1, + }, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table1", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case1File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 2.1: + // csv has two columns(colA, colB) with the header. + // tidb only has one column(colB). + // we expect the check failed. + { + nil, + "TiDB schema `db1`.`table2` doesn't have column cola,(.*)use tables.ignoreColumns to ignore(.*)", + 1, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB has the default value + Name: model.NewCIStr("colB"), + DefaultIsExpr: true, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 2.2: + // csv has two columns(colA, colB) with the header. + // tidb only has one column(colB). + // we ignore colA by set config tables.IgnoreColumns + // we expect the check success. + { + []*config.IgnoreColumns{ + { + DB: "db1", + Table: "table2", + Columns: []string{"cola"}, + }, + }, + "", + 0, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB has the default value + Name: model.NewCIStr("colB"), + DefaultIsExpr: true, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 2.3: + // csv has two columns(colA, colB) with the header. + // tidb has two columns(colB, colC). + // we ignore colA by set config tables.IgnoreColumns + // colC doesn't have the default value. + // we expect the check failed. + { + []*config.IgnoreColumns{ + { + DB: "db1", + Table: "table2", + Columns: []string{"cola"}, + }, + }, + "TiDB schema `db1`.`table2` doesn't have the default value for colc(.*)", + 1, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB has the default value + Name: model.NewCIStr("colB"), + DefaultIsExpr: true, + }, + { + // colC doesn't have the default value + Name: model.NewCIStr("colC"), + FieldType: types.FieldType{ + Flag: 1, + }, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 2.4: + // csv has two columns(colA, colB) with the header. + // tidb has two columns(colB, colC). + // we ignore colB by set config tables.IgnoreColumns + // colB doesn't have the default value. + // we expect the check failed. + { + []*config.IgnoreColumns{ + { + TableFilter: []string{"`db1`.`table2`"}, + Columns: []string{"colb"}, + }, + }, + "TiDB schema `db1`.`table2`'s column colb cannot be ignored(.*)", + 2, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB doesn't have the default value + Name: model.NewCIStr("colB"), + FieldType: types.FieldType{ + Flag: 1, + }, + }, + { + // colC has the default value + Name: model.NewCIStr("colC"), + DefaultIsExpr: true, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 3: + // table3's schema file not found. + // tidb has no table3. + // we expect the check failed. + { + []*config.IgnoreColumns{ + { + TableFilter: []string{"`db1`.`table2`"}, + Columns: []string{"colb"}, + }, + }, + "TiDB schema `db1`.`table3` doesn't exists(.*)", + 1, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "": {}, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table3", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }, + }, + // Case 4: + // table4 has two datafiles for table. we only check the first file. + // we expect the check success. + { + []*config.IgnoreColumns{ + { + DB: "db1", + Table: "table2", + Columns: []string{"cola"}, + }, + }, + "", + 0, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB has the default value + Name: model.NewCIStr("colB"), + DefaultIsExpr: true, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + // This type will make the check failed. + // but it's the second file for table. + // so it's unreachable so this case will success. + Type: mydump.SourceTypeIgnore, + }, + }, + }, + }, + }, + } + + for _, ca := range cases { + template := NewSimpleTemplate() + cfg := &config.Config{ + Mydumper: config.MydumperRuntime{ + ReadBlockSize: config.ReadBlockSize, + CSV: config.CSVConfig{ + Separator: ",", + Delimiter: `"`, + Header: ca.hasHeader, + NotNull: false, + Null: `\N`, + BackslashEscape: true, + TrimLastSep: false, + }, + IgnoreColumns: ca.ignoreColumns, + }, + } + rc := &Controller{ + cfg: cfg, + checkTemplate: template, + store: mockStore, + dbInfos: ca.dbInfos, + ioWorkers: worker.NewPool(context.Background(), 1, "io"), + } + msgs, err := rc.SchemaIsValid(ctx, ca.tableMeta) + require.NoError(s.T(), err) + require.Len(s.T(), msgs, ca.MsgNum) + if len(msgs) > 0 { + require.Regexp(s.T(), ca.expectMsg, msgs[0]) + } + } +} + +func (s *tableRestoreSuite) TestGBKEncodedSchemaIsValid() { + cfg := &config.Config{ + Mydumper: config.MydumperRuntime{ + ReadBlockSize: config.ReadBlockSize, + DataCharacterSet: "gb18030", + DataInvalidCharReplace: string(utf8.RuneError), + CSV: config.CSVConfig{ + Separator: ",", + Delimiter: `"`, + Header: true, + NotNull: false, + Null: `\N`, + BackslashEscape: true, + TrimLastSep: false, + }, + IgnoreColumns: nil, + }, + } + charsetConvertor, err := mydump.NewCharsetConvertor(cfg.Mydumper.DataCharacterSet, cfg.Mydumper.DataInvalidCharReplace) + require.NoError(s.T(), err) + dir := s.T().TempDir() + mockStore, err := storage.NewLocalStorage(dir) + require.NoError(s.T(), err) + csvContent, err := charsetConvertor.Encode(string([]byte("\"colA\",\"colB\"\n\"a\",\"b\""))) + require.NoError(s.T(), err) + ctx := context.Background() + csvFile := "db1.gbk_table.csv" + err = mockStore.WriteFile(ctx, csvFile, []byte(csvContent)) + require.NoError(s.T(), err) + + rc := &Controller{ + cfg: cfg, + checkTemplate: NewSimpleTemplate(), + store: mockStore, + dbInfos: map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "gbk_table": { + ID: 1, + DB: "db1", + Name: "gbk_table", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + Name: model.NewCIStr("colA"), + FieldType: types.FieldType{ + Flag: 1, + }, + }, + { + Name: model.NewCIStr("colB"), + FieldType: types.FieldType{ + Flag: 1, + }, + }, + }, + }, + }, + }, + }, + }, + ioWorkers: worker.NewPool(ctx, 1, "io"), + } + msgs, err := rc.SchemaIsValid(ctx, &mydump.MDTableMeta{ + DB: "db1", + Name: "gbk_table", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: csvFile, + Type: mydump.SourceTypeCSV, + }, + }, + }, + }) + require.NoError(s.T(), err) + require.Len(s.T(), msgs, 0) +} diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index 87484dda80348..fe77200124afb 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -265,13 +265,15 @@ func LoadSchemaInfo( if err != nil { return nil, errors.Trace(err) } + // Table names are case-sensitive in mydump.MDTableMeta. + // We should always use the original tbl.Name in checkpoints. tableInfo := &checkpoints.TidbTableInfo{ ID: tblInfo.ID, DB: schema.Name, - Name: tableName, + Name: tbl.Name, Core: tblInfo, } - dbInfo.Tables[tableName] = tableInfo + dbInfo.Tables[tbl.Name] = tableInfo } result[schema.Name] = dbInfo diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index 4599d64540d17..89a5b40433797 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -21,7 +21,6 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/go-sql-driver/mysql" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/glue" @@ -32,156 +31,120 @@ import ( "github.com/pingcap/tidb/parser/model" tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -var _ = Suite(&tidbSuite{}) - type tidbSuite struct { mockDB sqlmock.Sqlmock timgr *TiDBManager tiGlue glue.Glue } -func TestTiDB(t *testing.T) { - TestingT(t) -} - -func (s *tidbSuite) SetUpTest(c *C) { +func newTiDBSuite(t *testing.T) (*tidbSuite, func()) { + var s tidbSuite db, mock, err := sqlmock.New() - c.Assert(err, IsNil) + require.NoError(t, err) s.mockDB = mock defaultSQLMode, err := tmysql.GetSQLMode(tmysql.DefaultSQLMode) - c.Assert(err, IsNil) + require.NoError(t, err) s.timgr = NewTiDBManagerWithDB(db, defaultSQLMode) s.tiGlue = glue.NewExternalTiDBGlue(db, defaultSQLMode) + return &s, func() { + s.timgr.Close() + require.NoError(t, s.mockDB.ExpectationsWereMet()) + } } -func (s *tidbSuite) TearDownTest(c *C) { - s.timgr.Close() - c.Assert(s.mockDB.ExpectationsWereMet(), IsNil) -} +func TestCreateTableIfNotExistsStmt(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() -func (s *tidbSuite) TestCreateTableIfNotExistsStmt(c *C) { dbName := "testdb" createSQLIfNotExistsStmt := func(createTable, tableName string) []string { res, err := createIfNotExistsStmt(s.tiGlue.GetParser(), createTable, dbName, tableName) - c.Assert(err, IsNil) + require.NoError(t, err) return res } - c.Assert( - createSQLIfNotExistsStmt("CREATE DATABASE `foo` CHARACTER SET = utf8 COLLATE = utf8_general_ci;", ""), - DeepEquals, - []string{"CREATE DATABASE IF NOT EXISTS `testdb` CHARACTER SET = utf8 COLLATE = utf8_general_ci;"}, - ) + require.Equal(t, []string{"CREATE DATABASE IF NOT EXISTS `testdb` CHARACTER SET = utf8 COLLATE = utf8_general_ci;"}, + createSQLIfNotExistsStmt("CREATE DATABASE `foo` CHARACTER SET = utf8 COLLATE = utf8_general_ci;", "")) - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` TINYINT(1));", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` TINYINT(1));"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` TINYINT(1));", "foo")) - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE IF NOT EXISTS `foo`(`bar` TINYINT(1));", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` TINYINT(1));"}, + createSQLIfNotExistsStmt("CREATE TABLE IF NOT EXISTS `foo`(`bar` TINYINT(1));", "foo")) // case insensitive - c.Assert( - createSQLIfNotExistsStmt("/* cOmmEnt */ creAte tablE `fOo`(`bar` TinyinT(1));", "fOo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`fOo` (`bar` TINYINT(1));"}, - ) - - c.Assert( - createSQLIfNotExistsStmt("/* coMMenT */ crEatE tAble If not EXISts `FoO`(`bAR` tiNyInT(1));", "FoO"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`FoO` (`bAR` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`fOo` (`bar` TINYINT(1));"}, + createSQLIfNotExistsStmt("/* cOmmEnt */ creAte tablE `fOo`(`bar` TinyinT(1));", "fOo")) + + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`FoO` (`bAR` TINYINT(1));"}, + createSQLIfNotExistsStmt("/* coMMenT */ crEatE tAble If not EXISts `FoO`(`bAR` tiNyInT(1));", "FoO")) // only one "CREATE TABLE" is replaced - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE');", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE');"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE');"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE');", "foo")) // test clustered index consistency - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY CLUSTERED COMMENT 'CREATE TABLE');", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![clustered_index] CLUSTERED */ COMMENT 'CREATE TABLE');"}, - ) - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE', PRIMARY KEY (`bar`) NONCLUSTERED);", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE',PRIMARY KEY(`bar`) /*T![clustered_index] NONCLUSTERED */);"}, - ) - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY /*T![clustered_index] NONCLUSTERED */ COMMENT 'CREATE TABLE');", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![clustered_index] NONCLUSTERED */ COMMENT 'CREATE TABLE');"}, - ) - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE', PRIMARY KEY (`bar`) /*T![clustered_index] CLUSTERED */);", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE',PRIMARY KEY(`bar`) /*T![clustered_index] CLUSTERED */);"}, - ) - - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY AUTO_RANDOM(2) COMMENT 'CREATE TABLE');", "foo"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![auto_rand] AUTO_RANDOM(2) */ COMMENT 'CREATE TABLE');"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![clustered_index] CLUSTERED */ COMMENT 'CREATE TABLE');"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY CLUSTERED COMMENT 'CREATE TABLE');", "foo")) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE',PRIMARY KEY(`bar`) /*T![clustered_index] NONCLUSTERED */);"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE', PRIMARY KEY (`bar`) NONCLUSTERED);", "foo")) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![clustered_index] NONCLUSTERED */ COMMENT 'CREATE TABLE');"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY /*T![clustered_index] NONCLUSTERED */ COMMENT 'CREATE TABLE');", "foo")) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) COMMENT 'CREATE TABLE',PRIMARY KEY(`bar`) /*T![clustered_index] CLUSTERED */);"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) COMMENT 'CREATE TABLE', PRIMARY KEY (`bar`) /*T![clustered_index] CLUSTERED */);", "foo")) + + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`foo` (`bar` INT(1) PRIMARY KEY /*T![auto_rand] AUTO_RANDOM(2) */ COMMENT 'CREATE TABLE');"}, + createSQLIfNotExistsStmt("CREATE TABLE `foo`(`bar` INT(1) PRIMARY KEY AUTO_RANDOM(2) COMMENT 'CREATE TABLE');", "foo")) // upper case becomes shorter - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `ſ`(`ı` TINYINT(1));", "ſ"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ſ` (`ı` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ſ` (`ı` TINYINT(1));"}, + createSQLIfNotExistsStmt("CREATE TABLE `ſ`(`ı` TINYINT(1));", "ſ")) // upper case becomes longer - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `ɑ`(`ȿ` TINYINT(1));", "ɑ"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ɑ` (`ȿ` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ɑ` (`ȿ` TINYINT(1));"}, + createSQLIfNotExistsStmt("CREATE TABLE `ɑ`(`ȿ` TINYINT(1));", "ɑ")) // non-utf-8 - c.Assert( - createSQLIfNotExistsStmt("CREATE TABLE `\xcc\xcc\xcc`(`\xdd\xdd\xdd` TINYINT(1));", "\xcc\xcc\xcc"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`\xcc\xcc\xcc` (`???` TINYINT(1));"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`\xcc\xcc\xcc` (`???` TINYINT(1));"}, + createSQLIfNotExistsStmt("CREATE TABLE `\xcc\xcc\xcc`(`\xdd\xdd\xdd` TINYINT(1));", "\xcc\xcc\xcc")) // renaming a table - c.Assert( - createSQLIfNotExistsStmt("create table foo(x int);", "ba`r"), - DeepEquals, - []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ba``r` (`x` INT);"}, - ) + require.Equal(t, []string{"CREATE TABLE IF NOT EXISTS `testdb`.`ba``r` (`x` INT);"}, + createSQLIfNotExistsStmt("create table foo(x int);", "ba`r")) // conditional comments - c.Assert( + require.Equal(t, []string{ + "SET NAMES 'binary';", + "SET @@SESSION.`FOREIGN_KEY_CHECKS`=0;", + "CREATE TABLE IF NOT EXISTS `testdb`.`m` (`z` DOUBLE) ENGINE = InnoDB AUTO_INCREMENT = 8343230 DEFAULT CHARACTER SET = UTF8;", + }, createSQLIfNotExistsStmt(` /*!40101 SET NAMES binary*/; /*!40014 SET FOREIGN_KEY_CHECKS=0*/; CREATE TABLE x.y (z double) ENGINE=InnoDB AUTO_INCREMENT=8343230 DEFAULT CHARSET=utf8; - `, "m"), - DeepEquals, - []string{ - "SET NAMES 'binary';", - "SET @@SESSION.`FOREIGN_KEY_CHECKS`=0;", - "CREATE TABLE IF NOT EXISTS `testdb`.`m` (`z` DOUBLE) ENGINE = InnoDB AUTO_INCREMENT = 8343230 DEFAULT CHARACTER SET = UTF8;", - }, - ) + `, "m")) // create view - c.Assert( + require.Equal(t, []string{ + "SET NAMES 'binary';", + "DROP TABLE IF EXISTS `testdb`.`m`;", + "DROP VIEW IF EXISTS `testdb`.`m`;", + "SET @`PREV_CHARACTER_SET_CLIENT`=@@`character_set_client`;", + "SET @`PREV_CHARACTER_SET_RESULTS`=@@`character_set_results`;", + "SET @`PREV_COLLATION_CONNECTION`=@@`collation_connection`;", + "SET @@SESSION.`character_set_client`=`utf8`;", + "SET @@SESSION.`character_set_results`=`utf8`;", + "SET @@SESSION.`collation_connection`=`utf8_general_ci`;", + "CREATE ALGORITHM = UNDEFINED DEFINER = `root`@`192.168.198.178` SQL SECURITY DEFINER VIEW `testdb`.`m` (`s`) AS SELECT `s` FROM `db1`.`v1` WHERE `i`<2;", + "SET @@SESSION.`character_set_client`=@`PREV_CHARACTER_SET_CLIENT`;", + "SET @@SESSION.`character_set_results`=@`PREV_CHARACTER_SET_RESULTS`;", + "SET @@SESSION.`collation_connection`=@`PREV_COLLATION_CONNECTION`;", + }, createSQLIfNotExistsStmt(` /*!40101 SET NAMES binary*/; DROP TABLE IF EXISTS v2; @@ -196,27 +159,12 @@ func (s *tidbSuite) TestCreateTableIfNotExistsStmt(c *C) { SET character_set_client = @PREV_CHARACTER_SET_CLIENT; SET character_set_results = @PREV_CHARACTER_SET_RESULTS; SET collation_connection = @PREV_COLLATION_CONNECTION; - `, "m"), - DeepEquals, - []string{ - "SET NAMES 'binary';", - "DROP TABLE IF EXISTS `testdb`.`m`;", - "DROP VIEW IF EXISTS `testdb`.`m`;", - "SET @`PREV_CHARACTER_SET_CLIENT`=@@`character_set_client`;", - "SET @`PREV_CHARACTER_SET_RESULTS`=@@`character_set_results`;", - "SET @`PREV_COLLATION_CONNECTION`=@@`collation_connection`;", - "SET @@SESSION.`character_set_client`=`utf8`;", - "SET @@SESSION.`character_set_results`=`utf8`;", - "SET @@SESSION.`collation_connection`=`utf8_general_ci`;", - "CREATE ALGORITHM = UNDEFINED DEFINER = `root`@`192.168.198.178` SQL SECURITY DEFINER VIEW `testdb`.`m` (`s`) AS SELECT `s` FROM `db1`.`v1` WHERE `i`<2;", - "SET @@SESSION.`character_set_client`=@`PREV_CHARACTER_SET_CLIENT`;", - "SET @@SESSION.`character_set_results`=@`PREV_CHARACTER_SET_RESULTS`;", - "SET @@SESSION.`collation_connection`=@`PREV_COLLATION_CONNECTION`;", - }, - ) + `, "m")) } -func (s *tidbSuite) TestInitSchema(c *C) { +func TestInitSchema(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -240,10 +188,12 @@ func (s *tidbSuite) TestInitSchema(c *C) { "t2": "/*!40014 SET FOREIGN_KEY_CHECKS=0*/;CREATE TABLE `db`.`t2` (xx TEXT) AUTO_INCREMENT=11203;", }) s.mockDB.MatchExpectationsInOrder(true) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *tidbSuite) TestInitSchemaSyntaxError(c *C) { +func TestInitSchemaSyntaxError(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -255,10 +205,12 @@ func (s *tidbSuite) TestInitSchemaSyntaxError(c *C) { err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ "t1": "create table `t1` with invalid syntax;", }) - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *tidbSuite) TestInitSchemaErrorLost(c *C) { +func TestInitSchemaErrorLost(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -279,10 +231,12 @@ func (s *tidbSuite) TestInitSchemaErrorLost(c *C) { "t1": "create table `t1` (a int);", "t2": "create table t2 (a int primary key, b varchar(200));", }) - c.Assert(err, ErrorMatches, ".*Column length too big.*") + require.Regexp(t, ".*Column length too big.*", err.Error()) } -func (s *tidbSuite) TestInitSchemaUnsupportedSchemaError(c *C) { +func TestInitSchemaUnsupportedSchemaError(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -300,10 +254,12 @@ func (s *tidbSuite) TestInitSchemaUnsupportedSchemaError(c *C) { err := InitSchema(ctx, s.tiGlue, "db", map[string]string{ "t1": "create table `t1` (a VARCHAR(999999999));", }) - c.Assert(err, ErrorMatches, ".*Column length too big.*") + require.Regexp(t, ".*Column length too big.*", err.Error()) } -func (s *tidbSuite) TestDropTable(c *C) { +func TestDropTable(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -313,10 +269,12 @@ func (s *tidbSuite) TestDropTable(c *C) { ExpectClose() err := s.timgr.DropTable(ctx, "`db`.`table`") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *tidbSuite) TestLoadSchemaInfo(c *C) { +func TestLoadSchemaInfo(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() tableCntBefore := metric.ReadCounter(metric.TableCounter.WithLabelValues(metric.TableStatePending, metric.TableResultSuccess)) @@ -326,15 +284,16 @@ func (s *tidbSuite) TestLoadSchemaInfo(c *C) { "CREATE TABLE `t1` (`a` INT PRIMARY KEY);"+ "CREATE TABLE `t2` (`b` VARCHAR(20), `c` BOOL, KEY (`b`, `c`));"+ // an extra table that not exists in dbMetas - "CREATE TABLE `t3` (`d` VARCHAR(20), `e` BOOL);", + "CREATE TABLE `t3` (`d` VARCHAR(20), `e` BOOL);"+ + "CREATE TABLE `T4` (`f` BIGINT PRIMARY KEY);", "", "") - c.Assert(err, IsNil) + require.NoError(t, err) tableInfos := make([]*model.TableInfo, 0, len(nodes)) sctx := mock.NewContext() for i, node := range nodes { - c.Assert(node, FitsTypeOf, &ast.CreateTableStmt{}) + require.IsType(t, node, &ast.CreateTableStmt{}) info, err := ddl.MockTableInfo(sctx, node.(*ast.CreateTableStmt), int64(i+100)) - c.Assert(err, IsNil) + require.NoError(t, err) info.State = model.StatePublic tableInfos = append(tableInfos, info) } @@ -351,16 +310,20 @@ func (s *tidbSuite) TestLoadSchemaInfo(c *C) { DB: "db", Name: "t2", }, + { + DB: "db", + Name: "t4", + }, }, }, } loaded, err := LoadSchemaInfo(ctx, dbMetas, func(ctx context.Context, schema string) ([]*model.TableInfo, error) { - c.Assert(schema, Equals, "db") + require.Equal(t, "db", schema) return tableInfos, nil }) - c.Assert(err, IsNil) - c.Assert(loaded, DeepEquals, map[string]*checkpoints.TidbDBInfo{ + require.NoError(t, err) + require.Equal(t, map[string]*checkpoints.TidbDBInfo{ "db": { Name: "db", Tables: map[string]*checkpoints.TidbTableInfo{ @@ -376,25 +339,33 @@ func (s *tidbSuite) TestLoadSchemaInfo(c *C) { Name: "t2", Core: tableInfos[1], }, + "t4": { + ID: 103, + DB: "db", + Name: "t4", + Core: tableInfos[3], + }, }, }, - }) + }, loaded) tableCntAfter := metric.ReadCounter(metric.TableCounter.WithLabelValues(metric.TableStatePending, metric.TableResultSuccess)) - c.Assert(tableCntAfter-tableCntBefore, Equals, 2.0) + require.Equal(t, 3.0, tableCntAfter-tableCntBefore) } -func (s *tidbSuite) TestLoadSchemaInfoMissing(c *C) { +func TestLoadSchemaInfoMissing(t *testing.T) { ctx := context.Background() _, err := LoadSchemaInfo(ctx, []*mydump.MDDatabaseMeta{{Name: "asdjalsjdlas"}}, func(ctx context.Context, schema string) ([]*model.TableInfo, error) { return nil, errors.Errorf("[schema:1049]Unknown database '%s'", schema) }) - c.Assert(err, ErrorMatches, ".*Unknown database.*") + require.Regexp(t, ".*Unknown database.*", err.Error()) } -func (s *tidbSuite) TestGetGCLifetime(c *C) { +func TestGetGCLifetime(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -404,11 +375,13 @@ func (s *tidbSuite) TestGetGCLifetime(c *C) { ExpectClose() res, err := ObtainGCLifeTime(ctx, s.timgr.db) - c.Assert(err, IsNil) - c.Assert(res, Equals, "10m") + require.NoError(t, err) + require.Equal(t, "10m", res) } -func (s *tidbSuite) TestSetGCLifetime(c *C) { +func TestSetGCLifetime(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -419,10 +392,12 @@ func (s *tidbSuite) TestSetGCLifetime(c *C) { ExpectClose() err := UpdateGCLifeTime(ctx, s.timgr.db, "12m") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *tidbSuite) TestAlterAutoInc(c *C) { +func TestAlterAutoInc(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -432,10 +407,12 @@ func (s *tidbSuite) TestAlterAutoInc(c *C) { ExpectClose() err := AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *tidbSuite) TestAlterAutoRandom(c *C) { +func TestAlterAutoRandom(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -445,10 +422,12 @@ func (s *tidbSuite) TestAlterAutoRandom(c *C) { ExpectClose() err := AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *tidbSuite) TestObtainRowFormatVersionSucceed(c *C) { +func TestObtainRowFormatVersionSucceed(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -470,7 +449,7 @@ func (s *tidbSuite) TestObtainRowFormatVersionSucceed(c *C) { ExpectClose() sysVars := ObtainImportantVariables(ctx, s.tiGlue.GetSQLExecutor(), true) - c.Assert(sysVars, DeepEquals, map[string]string{ + require.Equal(t, map[string]string{ "tidb_row_format_version": "2", "max_allowed_packet": "1073741824", "div_precision_increment": "10", @@ -479,10 +458,12 @@ func (s *tidbSuite) TestObtainRowFormatVersionSucceed(c *C) { "default_week_format": "1", "block_encryption_mode": "aes-256-cbc", "group_concat_max_len": "1073741824", - }) + }, sysVars) } -func (s *tidbSuite) TestObtainRowFormatVersionFailure(c *C) { +func TestObtainRowFormatVersionFailure(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -496,7 +477,7 @@ func (s *tidbSuite) TestObtainRowFormatVersionFailure(c *C) { ExpectClose() sysVars := ObtainImportantVariables(ctx, s.tiGlue.GetSQLExecutor(), true) - c.Assert(sysVars, DeepEquals, map[string]string{ + require.Equal(t, map[string]string{ "tidb_row_format_version": "1", "max_allowed_packet": "67108864", "div_precision_increment": "4", @@ -505,10 +486,12 @@ func (s *tidbSuite) TestObtainRowFormatVersionFailure(c *C) { "default_week_format": "0", "block_encryption_mode": "aes-128-ecb", "group_concat_max_len": "1024", - }) + }, sysVars) } -func (s *tidbSuite) TestObtainNewCollationEnabled(c *C) { +func TestObtainNewCollationEnabled(t *testing.T) { + s, clean := newTiDBSuite(t) + defer clean() ctx := context.Background() s.mockDB. @@ -521,14 +504,14 @@ func (s *tidbSuite) TestObtainNewCollationEnabled(c *C) { ExpectQuery("\\QSELECT variable_value FROM mysql.tidb WHERE variable_name = 'new_collation_enabled'\\E"). WillReturnError(errors.New("mock permission deny")) _, err := ObtainNewCollationEnabled(ctx, s.tiGlue.GetSQLExecutor()) - c.Assert(err, ErrorMatches, "obtain new collation enabled failed: mock permission deny") + require.Equal(t, "obtain new collation enabled failed: mock permission deny", err.Error()) s.mockDB. ExpectQuery("\\QSELECT variable_value FROM mysql.tidb WHERE variable_name = 'new_collation_enabled'\\E"). WillReturnRows(sqlmock.NewRows([]string{"variable_value"}).RowError(0, sql.ErrNoRows)) version, err := ObtainNewCollationEnabled(ctx, s.tiGlue.GetSQLExecutor()) - c.Assert(err, IsNil) - c.Assert(version, Equals, false) + require.NoError(t, err) + require.Equal(t, false, version) kvMap := map[string]bool{ "True": true, @@ -540,8 +523,8 @@ func (s *tidbSuite) TestObtainNewCollationEnabled(c *C) { WillReturnRows(sqlmock.NewRows([]string{"variable_value"}).AddRow(k)) version, err = ObtainNewCollationEnabled(ctx, s.tiGlue.GetSQLExecutor()) - c.Assert(err, IsNil) - c.Assert(version, Equals, v) + require.NoError(t, err) + require.Equal(t, v, version) } s.mockDB. ExpectClose() diff --git a/br/pkg/lightning/web/progress.go b/br/pkg/lightning/web/progress.go index 0acbae865dd53..8a3412087b94f 100644 --- a/br/pkg/lightning/web/progress.go +++ b/br/pkg/lightning/web/progress.go @@ -8,6 +8,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "go.uber.org/atomic" ) // checkpointsMap is a concurrent map (table name → checkpoints). @@ -108,11 +109,25 @@ type taskProgress struct { checkpoints checkpointsMap } -var currentProgress = taskProgress{ - checkpoints: makeCheckpointsMap(), +var ( + currentProgress *taskProgress + // whether progress is enabled + progressEnabled = atomic.NewBool(false) +) + +// EnableCurrentProgress init current progress struct on demand. +// NOTE: this call is not thread safe, so it should only be inited once at the very beginning of progress start. +func EnableCurrentProgress() { + currentProgress = &taskProgress{ + checkpoints: makeCheckpointsMap(), + } + progressEnabled.Store(true) } func BroadcastStartTask() { + if !progressEnabled.Load() { + return + } currentProgress.mu.Lock() currentProgress.Status = taskStatusRunning currentProgress.mu.Unlock() @@ -121,6 +136,9 @@ func BroadcastStartTask() { } func BroadcastEndTask(err error) { + if !progressEnabled.Load() { + return + } errString := errors.ErrorStack(err) currentProgress.mu.Lock() @@ -130,6 +148,9 @@ func BroadcastEndTask(err error) { } func BroadcastInitProgress(databases []*mydump.MDDatabaseMeta) { + if !progressEnabled.Load() { + return + } tables := make(map[string]*tableInfo, len(databases)) for _, db := range databases { @@ -145,6 +166,9 @@ func BroadcastInitProgress(databases []*mydump.MDDatabaseMeta) { } func BroadcastTableCheckpoint(tableName string, cp *checkpoints.TableCheckpoint) { + if !progressEnabled.Load() { + return + } currentProgress.mu.Lock() currentProgress.Tables[tableName].Status = taskStatusRunning currentProgress.mu.Unlock() @@ -154,6 +178,9 @@ func BroadcastTableCheckpoint(tableName string, cp *checkpoints.TableCheckpoint) } func BroadcastCheckpointDiff(diffs map[string]*checkpoints.TableCheckpointDiff) { + if !progressEnabled.Load() { + return + } totalWrittens := currentProgress.checkpoints.update(diffs) currentProgress.mu.Lock() @@ -164,6 +191,9 @@ func BroadcastCheckpointDiff(diffs map[string]*checkpoints.TableCheckpointDiff) } func BroadcastError(tableName string, err error) { + if !progressEnabled.Load() { + return + } errString := errors.ErrorStack(err) currentProgress.mu.Lock() @@ -175,11 +205,17 @@ func BroadcastError(tableName string, err error) { } func MarshalTaskProgress() ([]byte, error) { + if !progressEnabled.Load() { + return nil, errors.New("progress is not enabled") + } currentProgress.mu.RLock() defer currentProgress.mu.RUnlock() return json.Marshal(¤tProgress) } func MarshalTableCheckpoints(tableName string) ([]byte, error) { + if !progressEnabled.Load() { + return nil, errors.New("progress is not enabled") + } return currentProgress.checkpoints.marshal(tableName) } diff --git a/br/pkg/membuf/buffer_test.go b/br/pkg/membuf/buffer_test.go index cc88119006a28..0d8e3ba06e609 100644 --- a/br/pkg/membuf/buffer_test.go +++ b/br/pkg/membuf/buffer_test.go @@ -80,8 +80,8 @@ func TestBufferIsolation(t *testing.T) { b1 := bytesBuf.AllocBytes(16) b2 := bytesBuf.AllocBytes(16) - require.Equal(t, len(b1), cap(b1)) - require.Equal(t, len(b2), cap(b2)) + require.Len(t, b1, cap(b1)) + require.Len(t, b2, cap(b2)) _, err := rand.Read(b2) require.NoError(t, err) diff --git a/br/pkg/metautil/metafile_test.go b/br/pkg/metautil/metafile_test.go index 6fe79a2b8e610..70ebe5df90c89 100644 --- a/br/pkg/metautil/metafile_test.go +++ b/br/pkg/metautil/metafile_test.go @@ -187,18 +187,18 @@ func TestEncryptAndDecrypt(t *testing.T) { if v.method == encryptionpb.EncryptionMethod_UNKNOWN { require.Error(t, err) } else if v.method == encryptionpb.EncryptionMethod_PLAINTEXT { - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, originalData, encryptData) decryptData, err := Decrypt(encryptData, &cipher, iv) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, decryptData, originalData) } else { - require.Nil(t, err) + require.NoError(t, err) require.NotEqual(t, originalData, encryptData) decryptData, err := Decrypt(encryptData, &cipher, iv) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, decryptData, originalData) wrongCipher := backuppb.CipherInfo{ @@ -209,7 +209,7 @@ func TestEncryptAndDecrypt(t *testing.T) { if len(v.rightKey) != len(v.wrongKey) { require.Error(t, err) } else { - require.Nil(t, err) + require.NoError(t, err) require.NotEqual(t, decryptData, originalData) } } diff --git a/br/pkg/mock/mock_cluster_test.go b/br/pkg/mock/mock_cluster_test.go index 01dde0c158901..740014f54efc7 100644 --- a/br/pkg/mock/mock_cluster_test.go +++ b/br/pkg/mock/mock_cluster_test.go @@ -14,7 +14,7 @@ func TestSmoke(t *testing.T) { defer goleak.VerifyNone( t, goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start")) m, err := mock.NewCluster() require.NoError(t, err) diff --git a/br/pkg/pdutil/main_test.go b/br/pkg/pdutil/main_test.go index 653d973bcd499..1c3cc2c34c21f 100644 --- a/br/pkg/pdutil/main_test.go +++ b/br/pkg/pdutil/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go index a3487187f8b09..5956c6b34417a 100644 --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -24,9 +24,9 @@ import ( berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/httputil" "github.com/pingcap/tidb/br/pkg/lightning/common" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/util/codec" pd "github.com/tikv/pd/client" - pdapi "github.com/tikv/pd/server/api" "go.uber.org/zap" "google.golang.org/grpc" ) @@ -350,12 +350,12 @@ func (p *PdController) getRegionCountWith( } // GetStoreInfo returns the info of store with the specified id. -func (p *PdController) GetStoreInfo(ctx context.Context, storeID uint64) (*pdapi.StoreInfo, error) { +func (p *PdController) GetStoreInfo(ctx context.Context, storeID uint64) (*pdtypes.StoreInfo, error) { return p.getStoreInfoWith(ctx, pdRequest, storeID) } func (p *PdController) getStoreInfoWith( - ctx context.Context, get pdHTTPRequest, storeID uint64) (*pdapi.StoreInfo, error) { + ctx context.Context, get pdHTTPRequest, storeID uint64) (*pdtypes.StoreInfo, error) { var err error for _, addr := range p.addrs { query := fmt.Sprintf( @@ -366,7 +366,7 @@ func (p *PdController) getStoreInfoWith( err = e continue } - store := pdapi.StoreInfo{} + store := pdtypes.StoreInfo{} err = json.Unmarshal(v, &store) if err != nil { return nil, errors.Trace(err) diff --git a/br/pkg/pdutil/pd_serial_test.go b/br/pkg/pdutil/pd_serial_test.go index c76e49cd70c61..05f0d34aa2ef2 100644 --- a/br/pkg/pdutil/pd_serial_test.go +++ b/br/pkg/pdutil/pd_serial_test.go @@ -17,12 +17,9 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" - "github.com/tikv/pd/pkg/typeutil" - "github.com/tikv/pd/server/api" - "github.com/tikv/pd/server/core" - "github.com/tikv/pd/server/statistics" ) func TestScheduler(t *testing.T) { @@ -106,26 +103,26 @@ func TestGetClusterVersion(t *testing.T) { } func TestRegionCount(t *testing.T) { - regions := core.NewRegionsInfo() - regions.SetRegion(core.NewRegionInfo(&metapb.Region{ + regions := &pdtypes.RegionTree{} + regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{ Id: 1, StartKey: codec.EncodeBytes(nil, []byte{1, 1}), EndKey: codec.EncodeBytes(nil, []byte{1, 3}), RegionEpoch: &metapb.RegionEpoch{}, }, nil)) - regions.SetRegion(core.NewRegionInfo(&metapb.Region{ + regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{ Id: 2, StartKey: codec.EncodeBytes(nil, []byte{1, 3}), EndKey: codec.EncodeBytes(nil, []byte{1, 5}), RegionEpoch: &metapb.RegionEpoch{}, }, nil)) - regions.SetRegion(core.NewRegionInfo(&metapb.Region{ + regions.SetRegion(pdtypes.NewRegionInfo(&metapb.Region{ Id: 3, StartKey: codec.EncodeBytes(nil, []byte{2, 3}), EndKey: codec.EncodeBytes(nil, []byte{3, 4}), RegionEpoch: &metapb.RegionEpoch{}, }, nil)) - require.Equal(t, 3, regions.Len()) + require.Equal(t, 3, len(regions.Regions)) mock := func( _ context.Context, addr string, prefix string, _ *http.Client, _ string, _ io.Reader, @@ -138,7 +135,7 @@ func TestRegionCount(t *testing.T) { t.Log(hex.EncodeToString([]byte(start))) t.Log(hex.EncodeToString([]byte(end))) scanRegions := regions.ScanRange([]byte(start), []byte(end), 0) - stats := statistics.RegionStats{Count: len(scanRegions)} + stats := pdtypes.RegionStats{Count: len(scanRegions)} ret, err := json.Marshal(stats) require.NoError(t, err) return ret, nil @@ -206,12 +203,12 @@ func TestPDRequestRetry(t *testing.T) { } func TestStoreInfo(t *testing.T) { - storeInfo := api.StoreInfo{ - Status: &api.StoreStatus{ - Capacity: typeutil.ByteSize(1024), - Available: typeutil.ByteSize(1024), + storeInfo := pdtypes.StoreInfo{ + Status: &pdtypes.StoreStatus{ + Capacity: pdtypes.ByteSize(1024), + Available: pdtypes.ByteSize(1024), }, - Store: &api.MetaStore{ + Store: &pdtypes.MetaStore{ StateName: "Tombstone", }, } diff --git a/br/pkg/pdutil/utils.go b/br/pkg/pdutil/utils.go index c07171669bc11..1ea6e8e0d113d 100644 --- a/br/pkg/pdutil/utils.go +++ b/br/pkg/pdutil/utils.go @@ -15,9 +15,9 @@ import ( "github.com/pingcap/errors" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/tablecodec" - "github.com/tikv/pd/pkg/codec" - "github.com/tikv/pd/server/schedule/placement" + "github.com/pingcap/tidb/util/codec" ) // UndoFunc is a 'undo' operation of some undoable command. @@ -65,7 +65,7 @@ func ResetTS(ctx context.Context, pdAddr string, ts uint64, tlsConf *tls.Config) } // GetPlacementRules return the current placement rules. -func GetPlacementRules(ctx context.Context, pdAddr string, tlsConf *tls.Config) ([]placement.Rule, error) { +func GetPlacementRules(ctx context.Context, pdAddr string, tlsConf *tls.Config) ([]pdtypes.Rule, error) { cli := httputil.NewClient(tlsConf) prefix := "http://" if tlsConf != nil { @@ -87,12 +87,12 @@ func GetPlacementRules(ctx context.Context, pdAddr string, tlsConf *tls.Config) return nil, errors.Trace(err) } if resp.StatusCode == http.StatusPreconditionFailed { - return []placement.Rule{}, nil + return []pdtypes.Rule{}, nil } if resp.StatusCode != http.StatusOK { return nil, errors.Annotatef(berrors.ErrPDInvalidResponse, "get placement rules failed: resp=%v, err=%v, code=%d", buf.String(), err, resp.StatusCode) } - var rules []placement.Rule + var rules []pdtypes.Rule err = json.Unmarshal(buf.Bytes(), &rules) if err != nil { return nil, errors.Trace(err) @@ -101,13 +101,13 @@ func GetPlacementRules(ctx context.Context, pdAddr string, tlsConf *tls.Config) } // SearchPlacementRule returns the placement rule matched to the table or nil. -func SearchPlacementRule(tableID int64, placementRules []placement.Rule, role placement.PeerRoleType) *placement.Rule { +func SearchPlacementRule(tableID int64, placementRules []pdtypes.Rule, role pdtypes.PeerRoleType) *pdtypes.Rule { for _, rule := range placementRules { key, err := hex.DecodeString(rule.StartKeyHex) if err != nil { continue } - _, decoded, err := codec.DecodeBytes(key) + _, decoded, err := codec.DecodeBytes(key, nil) if err != nil { continue } diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 7c6f2bbf09f90..18c359fc203c0 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -38,11 +38,11 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" - "github.com/tikv/pd/server/schedule/placement" "go.uber.org/zap" "golang.org/x/sync/errgroup" "google.golang.org/grpc" @@ -65,8 +65,10 @@ type Client struct { tlsConf *tls.Config keepaliveConf keepalive.ClientParameters - databases map[string]*utils.Database - ddlJobs []*model.Job + databases map[string]*utils.Database + ddlJobs []*model.Job + // ddlJobsMap record the tables' auto id need to restore after create table + ddlJobsMap map[UniqueTableName]bool backupMeta *backuppb.BackupMeta // TODO Remove this field or replace it with a []*DB, // since https://github.com/pingcap/br/pull/377 needs more DBs to speed up DDL execution. @@ -323,8 +325,8 @@ func (rc *Client) ResetTS(ctx context.Context, pdAddrs []string) error { } // GetPlacementRules return the current placement rules. -func (rc *Client) GetPlacementRules(ctx context.Context, pdAddrs []string) ([]placement.Rule, error) { - var placementRules []placement.Rule +func (rc *Client) GetPlacementRules(ctx context.Context, pdAddrs []string) ([]pdtypes.Rule, error) { + var placementRules []pdtypes.Rule i := 0 errRetry := utils.WithRetry(ctx, func() error { var err error @@ -424,7 +426,7 @@ func (rc *Client) createTables( if rc.IsSkipCreateSQL() { log.Info("skip create table and alter autoIncID") } else { - err := db.CreateTables(ctx, tables) + err := db.CreateTables(ctx, tables, rc.GetDDLJobsMap()) if err != nil { return nil, errors.Trace(err) } @@ -460,12 +462,11 @@ func (rc *Client) createTable( dom *domain.Domain, table *metautil.Table, newTS uint64, - ddlTables map[UniqueTableName]bool, ) (CreatedTable, error) { if rc.IsSkipCreateSQL() { log.Info("skip create table and alter autoIncID", zap.Stringer("table", table.Info.Name)) } else { - err := db.CreateTable(ctx, table, ddlTables) + err := db.CreateTable(ctx, table, rc.GetDDLJobsMap()) if err != nil { return CreatedTable{}, errors.Trace(err) } @@ -504,7 +505,7 @@ func (rc *Client) GoCreateTables( // Could we have a smaller size of tables? log.Info("start create tables") - ddlTables := rc.DDLJobsMap() + rc.GenerateDDLJobsMap() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("Client.GoCreateTables", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -540,7 +541,7 @@ func (rc *Client) GoCreateTables( default: } - rt, err := rc.createTable(c, db, dom, t, newTS, ddlTables) + rt, err := rc.createTable(c, db, dom, t, newTS) if err != nil { log.Error("create table failed", zap.Error(err), @@ -748,7 +749,7 @@ func (rc *Client) RestoreFiles( zap.Duration("take", time.Since(fileStart))) updateCh.Inc() }() - return rc.fileImporter.Import(ectx, filesReplica, rewriteRules, rc.cipher) + return rc.fileImporter.Import(ectx, filesReplica, rewriteRules, rc.cipher, rc.backupMeta.ApiVersion) }) } @@ -789,7 +790,7 @@ func (rc *Client) RestoreRaw( rc.workerPool.ApplyOnErrorGroup(eg, func() error { defer updateCh.Inc() - return rc.fileImporter.Import(ectx, []*backuppb.File{fileReplica}, EmptyRewriteRule(), rc.cipher) + return rc.fileImporter.Import(ectx, []*backuppb.File{fileReplica}, EmptyRewriteRule(), rc.cipher, rc.backupMeta.ApiVersion) }) } if err := eg.Wait(); err != nil { @@ -1112,7 +1113,7 @@ func (rc *Client) SetupPlacementRules(ctx context.Context, tables []*model.Table } rule.Index = 100 rule.Override = true - rule.LabelConstraints = append(rule.LabelConstraints, placement.LabelConstraint{ + rule.LabelConstraints = append(rule.LabelConstraints, pdtypes.LabelConstraint{ Key: restoreLabelKey, Op: "in", Values: []string{restoreLabelValue}, @@ -1230,22 +1231,25 @@ func (rc *Client) IsSkipCreateSQL() bool { return rc.noSchema } -// DDLJobsMap returns a map[UniqueTableName]bool about < db table, hasCreate/hasTruncate DDL >. +// GenerateDDLJobsMap returns a map[UniqueTableName]bool about < db table, hasCreate/hasTruncate DDL >. // if we execute some DDLs before create table. // we may get two situation that need to rebase auto increment/random id. // 1. truncate table: truncate will generate new id cache. // 2. create table/create and rename table: the first create table will lock down the id cache. // because we cannot create onExistReplace table. // so the final create DDL with the correct auto increment/random id won't be executed. -func (rc *Client) DDLJobsMap() map[UniqueTableName]bool { - m := make(map[UniqueTableName]bool) +func (rc *Client) GenerateDDLJobsMap() { + rc.ddlJobsMap = make(map[UniqueTableName]bool) for _, job := range rc.ddlJobs { switch job.Type { case model.ActionTruncateTable, model.ActionCreateTable, model.ActionRenameTable: - m[UniqueTableName{job.SchemaName, job.BinlogInfo.TableInfo.Name.String()}] = true + rc.ddlJobsMap[UniqueTableName{job.SchemaName, job.BinlogInfo.TableInfo.Name.String()}] = true } } - return m +} + +func (rc *Client) GetDDLJobsMap() map[UniqueTableName]bool { + return rc.ddlJobsMap } // PreCheckTableTiFlashReplica checks whether TiFlash replica is less than TiFlash node. diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index 8a9f0681c5b97..f4ef5dc8256be 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -182,38 +182,10 @@ func (db *DB) restoreSequence(ctx context.Context, table *metautil.Table) error return errors.Trace(err) } -// CreateTables execute a internal CREATE TABLES. -func (db *DB) CreateTables(ctx context.Context, tables []*metautil.Table) error { - if batchSession, ok := db.se.(glue.BatchCreateTableSession); ok { - m := map[string][]*model.TableInfo{} - for _, table := range tables { - m[table.DB.Name.L] = append(m[table.DB.Name.L], table.Info) - } - if err := batchSession.CreateTables(ctx, m); err != nil { - return err - } - - for _, table := range tables { - err := db.restoreSequence(ctx, table) - return err - } - } - - return nil -} - -// CreateTable executes a CREATE TABLE SQL. -func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, ddlTables map[UniqueTableName]bool) error { - err := db.se.CreateTable(ctx, table.DB.Name, table.Info) - if err != nil { - log.Error("create table failed", - zap.Stringer("db", table.DB.Name), - zap.Stringer("table", table.Info.Name), - zap.Error(err)) - return errors.Trace(err) - } +func (db *DB) CreateTablePostRestore(ctx context.Context, table *metautil.Table, ddlTables map[UniqueTableName]bool) error { var restoreMetaSQL string + var err error switch { case table.Info.IsView(): return nil @@ -252,7 +224,47 @@ func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, ddlTables return errors.Trace(err) } } - return errors.Trace(err) + return nil +} + +// CreateTables execute a internal CREATE TABLES. +func (db *DB) CreateTables(ctx context.Context, tables []*metautil.Table, ddlTables map[UniqueTableName]bool) error { + if batchSession, ok := db.se.(glue.BatchCreateTableSession); ok { + m := map[string][]*model.TableInfo{} + for _, table := range tables { + m[table.DB.Name.L] = append(m[table.DB.Name.L], table.Info) + } + if err := batchSession.CreateTables(ctx, m); err != nil { + return err + } + + for _, table := range tables { + err := db.CreateTablePostRestore(ctx, table, ddlTables) + if err != nil { + return errors.Trace(err) + } + } + } + return nil +} + +// CreateTable executes a CREATE TABLE SQL. +func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, ddlTables map[UniqueTableName]bool) error { + err := db.se.CreateTable(ctx, table.DB.Name, table.Info) + if err != nil { + log.Error("create table failed", + zap.Stringer("db", table.DB.Name), + zap.Stringer("table", table.Info.Name), + zap.Error(err)) + return errors.Trace(err) + } + + err = db.CreateTablePostRestore(ctx, table, ddlTables) + if err != nil { + return errors.Trace(err) + } + + return err } // Close closes the connection. diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index 894ed1fd94179..1393ff2e66dbf 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -138,6 +138,7 @@ func TestCreateTablesInDb(t *testing.T) { tables := make([]*metautil.Table, 4) intField := types.NewFieldType(mysql.TypeLong) intField.Charset = "binary" + ddlJobMap := make(map[restore.UniqueTableName]bool) for i := len(tables) - 1; i >= 0; i-- { tables[i] = &metautil.Table{ DB: dbSchema, @@ -154,12 +155,13 @@ func TestCreateTablesInDb(t *testing.T) { Collate: "utf8mb4_bin", }, } + ddlJobMap[restore.UniqueTableName{dbSchema.Name.String(), tables[i].Info.Name.String()}] = false } db, err := restore.NewDB(gluetidb.New(), s.mock.Storage) - require.Nil(t, err) + require.NoError(t, err) - err = db.CreateTables(context.Background(), tables) - require.Nil(t, err) + err = db.CreateTables(context.Background(), tables, ddlJobMap) + require.NoError(t, err) } diff --git a/br/pkg/restore/import.go b/br/pkg/restore/import.go index aafb144959202..39068d6725502 100644 --- a/br/pkg/restore/import.go +++ b/br/pkg/restore/import.go @@ -266,6 +266,7 @@ func (importer *FileImporter) Import( files []*backuppb.File, rewriteRules *RewriteRules, cipher *backuppb.CipherInfo, + apiVersion kvrpcpb.APIVersion, ) error { start := time.Now() log.Debug("import file", logutil.Files(files)) @@ -317,7 +318,7 @@ func (importer *FileImporter) Import( for i, f := range remainFiles { var downloadMeta *import_sstpb.SSTMeta if importer.isRawKvMode { - downloadMeta, e = importer.downloadRawKVSST(ctx, info, f, cipher) + downloadMeta, e = importer.downloadRawKVSST(ctx, info, f, cipher, apiVersion) } else { downloadMeta, e = importer.downloadSST(ctx, info, f, rewriteRules, cipher) } @@ -528,6 +529,7 @@ func (importer *FileImporter) downloadRawKVSST( regionInfo *RegionInfo, file *backuppb.File, cipher *backuppb.CipherInfo, + apiVersion kvrpcpb.APIVersion, ) (*import_sstpb.SSTMeta, error) { uid := uuid.New() id := uid[:] @@ -586,6 +588,7 @@ func (importer *FileImporter) downloadRawKVSST( downloadResp := atomicResp.Load().(*import_sstpb.DownloadResponse) sstMeta.Range.Start = downloadResp.Range.GetStart() sstMeta.Range.End = downloadResp.Range.GetEnd() + sstMeta.ApiVersion = apiVersion return &sstMeta, nil } diff --git a/br/pkg/restore/main_test.go b/br/pkg/restore/main_test.go index 0b2cea2e47030..dd8a40f10d779 100644 --- a/br/pkg/restore/main_test.go +++ b/br/pkg/restore/main_test.go @@ -28,7 +28,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/br/pkg/restore/range.go b/br/pkg/restore/range.go index ac512de757df1..81881e78e8182 100644 --- a/br/pkg/restore/range.go +++ b/br/pkg/restore/range.go @@ -75,8 +75,10 @@ func SortRanges(ranges []rtree.Range, rewriteRules *RewriteRules) ([]rtree.Range // RegionInfo includes a region and the leader of the region. type RegionInfo struct { - Region *metapb.Region - Leader *metapb.Peer + Region *metapb.Region + Leader *metapb.Peer + PendingPeers []*metapb.Peer + DownPeers []*metapb.Peer } // ContainsInterior returns whether the region contains the given key, and also diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index 87bc2183e699e..e054190a41bae 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/tikv/pd/pkg/codec" + "github.com/pingcap/tidb/util/codec" "go.uber.org/multierr" "go.uber.org/zap" "google.golang.org/grpc/codes" @@ -96,8 +96,8 @@ func (rs *RegionSplitter) Split( if errSplit != nil { return errors.Trace(errSplit) } - minKey := codec.EncodeBytes(sortedRanges[0].StartKey) - maxKey := codec.EncodeBytes(sortedRanges[len(sortedRanges)-1].EndKey) + minKey := codec.EncodeBytes(nil, sortedRanges[0].StartKey) + maxKey := codec.EncodeBytes(nil, sortedRanges[len(sortedRanges)-1].EndKey) interval := SplitRetryInterval scatterRegions := make([]*RegionInfo, 0) SplitRegions: @@ -131,7 +131,7 @@ SplitRegions: log.Error("split regions no valid key", logutil.Key("startKey", region.Region.StartKey), logutil.Key("endKey", region.Region.EndKey), - logutil.Key("key", codec.EncodeBytes(key)), + logutil.Key("key", codec.EncodeBytes(nil, key)), rtree.ZapRanges(ranges)) } return errors.Trace(errSplit) @@ -185,12 +185,27 @@ SplitRegions: return nil } -func (rs *RegionSplitter) hasRegion(ctx context.Context, regionID uint64) (bool, error) { +func (rs *RegionSplitter) hasHealthyRegion(ctx context.Context, regionID uint64) (bool, error) { regionInfo, err := rs.client.GetRegionByID(ctx, regionID) if err != nil { return false, errors.Trace(err) } - return regionInfo != nil, nil + // the region hasn't get ready. + if regionInfo == nil { + return false, nil + } + + // check whether the region is healthy and report. + // TODO: the log may be too verbose. we should use Prometheus metrics once it get ready for BR. + for _, peer := range regionInfo.PendingPeers { + log.Debug("unhealthy region detected", logutil.Peer(peer), zap.String("type", "pending")) + } + for _, peer := range regionInfo.DownPeers { + log.Debug("unhealthy region detected", logutil.Peer(peer), zap.String("type", "down")) + } + // we ignore down peers for they are (normally) hard to be fixed in reasonable time. + // (or once there is a peer down, we may get stuck at waiting region get ready.) + return len(regionInfo.PendingPeers) == 0, nil } func (rs *RegionSplitter) isScatterRegionFinished(ctx context.Context, regionID uint64) (bool, error) { @@ -218,7 +233,7 @@ func (rs *RegionSplitter) isScatterRegionFinished(ctx context.Context, regionID func (rs *RegionSplitter) waitForSplit(ctx context.Context, regionID uint64) { interval := SplitCheckInterval for i := 0; i < SplitCheckMaxRetryTimes; i++ { - ok, err := rs.hasRegion(ctx, regionID) + ok, err := rs.hasHealthyRegion(ctx, regionID) if err != nil { log.Warn("wait for split failed", zap.Error(err)) return @@ -508,7 +523,7 @@ func NeedSplit(splitKey []byte, regions []*RegionInfo) *RegionInfo { if len(splitKey) == 0 { return nil } - splitKey = codec.EncodeBytes(splitKey) + splitKey = codec.EncodeBytes(nil, splitKey) for _, region := range regions { // If splitKey is the boundary of the region if bytes.Equal(splitKey, region.Region.GetStartKey()) { diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index ed24fc3984a52..c23aba54f2132 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -28,9 +28,8 @@ import ( berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/httputil" "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/store/pdtypes" pd "github.com/tikv/pd/client" - "github.com/tikv/pd/server/config" - "github.com/tikv/pd/server/schedule/placement" "go.uber.org/multierr" "go.uber.org/zap" "google.golang.org/grpc" @@ -68,9 +67,9 @@ type SplitClient interface { // Limit limits the maximum number of regions returned. ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*RegionInfo, error) // GetPlacementRule loads a placement rule from PD. - GetPlacementRule(ctx context.Context, groupID, ruleID string) (placement.Rule, error) + GetPlacementRule(ctx context.Context, groupID, ruleID string) (pdtypes.Rule, error) // SetPlacementRule insert or update a placement rule to PD. - SetPlacementRule(ctx context.Context, rule placement.Rule) error + SetPlacementRule(ctx context.Context, rule pdtypes.Rule) error // DeletePlacementRule removes a placement rule from PD. DeletePlacementRule(ctx context.Context, groupID, ruleID string) error // SetStoreLabel add or update specified label of stores. If labelValue @@ -172,8 +171,10 @@ func (c *pdClient) GetRegionByID(ctx context.Context, regionID uint64) (*RegionI return nil, nil } return &RegionInfo{ - Region: region.Meta, - Leader: region.Leader, + Region: region.Meta, + Leader: region.Leader, + PendingPeers: region.PendingPeers, + DownPeers: region.DownPeers, }, nil } @@ -427,7 +428,7 @@ func (c *pdClient) getStoreCount(ctx context.Context) (int, error) { func (c *pdClient) getMaxReplica(ctx context.Context) (int, error) { api := c.getPDAPIAddr() - configAPI := api + "/pd/api/v1/config" + configAPI := api + "/pd/api/v1/config/replicate" req, err := http.NewRequestWithContext(ctx, "GET", configAPI, nil) if err != nil { return 0, errors.Trace(err) @@ -441,11 +442,11 @@ func (c *pdClient) getMaxReplica(ctx context.Context) (int, error) { log.Error("Response fail to close", zap.Error(err)) } }() - var conf config.Config + var conf pdtypes.ReplicationConfig if err := json.NewDecoder(res.Body).Decode(&conf); err != nil { return 0, errors.Trace(err) } - return int(conf.Replication.MaxReplicas), nil + return int(conf.MaxReplicas), nil } func (c *pdClient) checkNeedScatter(ctx context.Context) (bool, error) { @@ -493,8 +494,8 @@ func (c *pdClient) ScanRegions(ctx context.Context, key, endKey []byte, limit in return regionInfos, nil } -func (c *pdClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (placement.Rule, error) { - var rule placement.Rule +func (c *pdClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (pdtypes.Rule, error) { + var rule pdtypes.Rule addr := c.getPDAPIAddr() if addr == "" { return rule, errors.Annotate(berrors.ErrRestoreSplitFailed, "failed to add stores labels: no leader") @@ -523,7 +524,7 @@ func (c *pdClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) return rule, nil } -func (c *pdClient) SetPlacementRule(ctx context.Context, rule placement.Rule) error { +func (c *pdClient) SetPlacementRule(ctx context.Context, rule pdtypes.Rule) error { addr := c.getPDAPIAddr() if addr == "" { return errors.Annotate(berrors.ErrPDLeaderNotFound, "failed to add stores labels") @@ -594,10 +595,10 @@ func (c *pdClient) getPDAPIAddr() string { return strings.TrimRight(addr, "/") } -func checkRegionEpoch(new, old *RegionInfo) bool { - return new.Region.GetId() == old.Region.GetId() && - new.Region.GetRegionEpoch().GetVersion() == old.Region.GetRegionEpoch().GetVersion() && - new.Region.GetRegionEpoch().GetConfVer() == old.Region.GetRegionEpoch().GetConfVer() +func checkRegionEpoch(_new, _old *RegionInfo) bool { + return _new.Region.GetId() == _old.Region.GetId() && + _new.Region.GetRegionEpoch().GetVersion() == _old.Region.GetRegionEpoch().GetVersion() && + _new.Region.GetRegionEpoch().GetConfVer() == _old.Region.GetRegionEpoch().GetConfVer() } // exponentialBackoffer trivially retry any errors it meets. diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index 7d85ac66c7711..141affd35d97f 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -16,10 +16,9 @@ import ( "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/util/codec" "github.com/stretchr/testify/require" - "github.com/tikv/pd/server/core" - "github.com/tikv/pd/server/schedule/placement" "go.uber.org/multierr" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -29,7 +28,7 @@ type TestClient struct { mu sync.RWMutex stores map[uint64]*metapb.Store regions map[uint64]*restore.RegionInfo - regionsInfo *core.RegionsInfo // For now it's only used in ScanRegions + regionsInfo *pdtypes.RegionTree // For now it's only used in ScanRegions nextRegionID uint64 injectInScatter func(*restore.RegionInfo) error supportBatchScatter bool @@ -42,9 +41,9 @@ func NewTestClient( regions map[uint64]*restore.RegionInfo, nextRegionID uint64, ) *TestClient { - regionsInfo := core.NewRegionsInfo() + regionsInfo := &pdtypes.RegionTree{} for _, regionInfo := range regions { - regionsInfo.SetRegion(core.NewRegionInfo(regionInfo.Region, regionInfo.Leader)) + regionsInfo.SetRegion(pdtypes.NewRegionInfo(regionInfo.Region, regionInfo.Leader)) } return &TestClient{ stores: stores, @@ -215,18 +214,18 @@ func (c *TestClient) ScanRegions(ctx context.Context, key, endKey []byte, limit regions := make([]*restore.RegionInfo, 0, len(infos)) for _, info := range infos { regions = append(regions, &restore.RegionInfo{ - Region: info.GetMeta(), - Leader: info.GetLeader(), + Region: info.Meta, + Leader: info.Leader, }) } return regions, nil } -func (c *TestClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r placement.Rule, err error) { +func (c *TestClient) GetPlacementRule(ctx context.Context, groupID, ruleID string) (r pdtypes.Rule, err error) { return } -func (c *TestClient) SetPlacementRule(ctx context.Context, rule placement.Rule) error { +func (c *TestClient) SetPlacementRule(ctx context.Context, rule pdtypes.Rule) error { return nil } diff --git a/br/pkg/storage/s3_test.go b/br/pkg/storage/s3_test.go index 9ea80fdebf830..fa4c858659ff7 100644 --- a/br/pkg/storage/s3_test.go +++ b/br/pkg/storage/s3_test.go @@ -971,7 +971,7 @@ func TestWalkDir(t *testing.T) { }, ) require.NoError(t, err) - require.Equal(t, len(contents), i) + require.Len(t, contents, i) // test with empty subDir i = 0 @@ -986,7 +986,7 @@ func TestWalkDir(t *testing.T) { }, ) require.NoError(t, err) - require.Equal(t, len(contents), i) + require.Len(t, contents, i) } // TestWalkDirBucket checks WalkDir retrieves all directory content under a bucket. @@ -1058,7 +1058,7 @@ func TestWalkDirWithEmptyPrefix(t *testing.T) { }, ) require.NoError(t, err) - require.Equal(t, len(contents), i) + require.Len(t, contents, i) // test with non-empty sub-dir i = 0 diff --git a/br/pkg/storage/writer_test.go b/br/pkg/storage/writer_test.go index bd99b218dde5a..c3d4080123f4f 100644 --- a/br/pkg/storage/writer_test.go +++ b/br/pkg/storage/writer_test.go @@ -35,7 +35,7 @@ func TestExternalFileWriter(t *testing.T) { p := []byte(str) written, err2 := writer.Write(ctx, p) require.Nil(t, err2) - require.Equal(t, len(p), written) + require.Len(t, p, written) } err = writer.Close(ctx) require.NoError(t, err) @@ -110,7 +110,7 @@ func TestCompressReaderWriter(t *testing.T) { p := []byte(str) written, err2 := writer.Write(ctx, p) require.Nil(t, err2) - require.Equal(t, len(p), written) + require.Len(t, p, written) } err = writer.Close(ctx) require.NoError(t, err) diff --git a/br/pkg/task/backup_raw.go b/br/pkg/task/backup_raw.go index febe151218706..3a4f29720c532 100644 --- a/br/pkg/task/backup_raw.go +++ b/br/pkg/task/backup_raw.go @@ -228,6 +228,7 @@ func RunBackupRaw(c context.Context, g glue.Glue, cmdName string, cfg *RawKvConf m.ClusterId = req.ClusterId m.ClusterVersion = clusterVersion m.BrVersion = brVersion + m.ApiVersion = client.GetApiVersion() }) err = metaWriter.FinishWriteMetas(ctx, metautil.AppendDataFile) if err != nil { diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 357c7d267e449..6d2b63571d5da 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -31,7 +31,7 @@ import ( "github.com/spf13/cobra" "github.com/spf13/pflag" pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/pkg/transport" + "go.etcd.io/etcd/client/pkg/v3/transport" "go.uber.org/zap" "google.golang.org/grpc/keepalive" ) diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index b4ecfc69c7c6a..aef4a17eb07ad 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -47,7 +47,7 @@ const ( defaultPDConcurrency = 1 defaultBatchFlushInterval = 16 * time.Second defaultDDLConcurrency = 16 - defaultFlagDdlBatchSize = 1 + defaultFlagDdlBatchSize = 128 ) // RestoreCommonConfig is the common configuration for all BR restore tasks. @@ -193,6 +193,37 @@ func (cfg *RestoreConfig) adjustRestoreConfig() { } } +func configureRestoreClient(ctx context.Context, client *restore.Client, cfg *RestoreConfig) error { + u, err := storage.ParseBackend(cfg.Storage, &cfg.BackendOptions) + if err != nil { + return errors.Trace(err) + } + opts := storage.ExternalStorageOptions{ + NoCredentials: cfg.NoCreds, + SendCredentials: cfg.SendCreds, + } + if err = client.SetStorage(ctx, u, &opts); err != nil { + return errors.Trace(err) + } + client.SetRateLimit(cfg.RateLimit) + client.SetCrypter(&cfg.CipherInfo) + client.SetConcurrency(uint(cfg.Concurrency)) + if cfg.Online { + client.EnableOnline() + } + if cfg.NoSchema { + client.EnableSkipCreateSQL() + } + client.SetSwitchModeInterval(cfg.SwitchModeInterval) + client.SetBatchDdlSize(cfg.DdlBatchSize) + err = client.LoadRestoreStores(ctx) + if err != nil { + return errors.Trace(err) + } + + return nil +} + // CheckRestoreDBAndTable is used to check whether the restore dbs or tables have been backup func CheckRestoreDBAndTable(client *restore.Client, cfg *RestoreConfig) error { if len(cfg.Schemas) == 0 && len(cfg.Tables) == 0 { @@ -244,13 +275,13 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf // Restore needs domain to do DDL. needDomain := true - mgr, err := NewMgr(ctx, g, cfg.PD, cfg.TLS, GetKeepalive(&cfg.Config), cfg.CheckRequirements, needDomain) + keepaliveCfg := GetKeepalive(&cfg.Config) + mgr, err := NewMgr(ctx, g, cfg.PD, cfg.TLS, keepaliveCfg, cfg.CheckRequirements, needDomain) if err != nil { return errors.Trace(err) } defer mgr.Close() - keepaliveCfg := GetKeepalive(&cfg.Config) keepaliveCfg.PermitWithoutStream = true client, err := restore.NewRestoreClient(g, mgr.GetPDClient(), mgr.GetStorage(), mgr.GetTLSConfig(), keepaliveCfg) if err != nil { @@ -258,29 +289,7 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf } defer client.Close() - u, err := storage.ParseBackend(cfg.Storage, &cfg.BackendOptions) - if err != nil { - return errors.Trace(err) - } - opts := storage.ExternalStorageOptions{ - NoCredentials: cfg.NoCreds, - SendCredentials: cfg.SendCreds, - } - if err = client.SetStorage(ctx, u, &opts); err != nil { - return errors.Trace(err) - } - client.SetRateLimit(cfg.RateLimit) - client.SetCrypter(&cfg.CipherInfo) - client.SetConcurrency(uint(cfg.Concurrency)) - if cfg.Online { - client.EnableOnline() - } - if cfg.NoSchema { - client.EnableSkipCreateSQL() - } - client.SetSwitchModeInterval(cfg.SwitchModeInterval) - client.SetBatchDdlSize(cfg.DdlBatchSize) - err = client.LoadRestoreStores(ctx) + err = configureRestoreClient(ctx, client, cfg) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/task/restore_test.go b/br/pkg/task/restore_test.go index 8b583dcf66e95..218033859f828 100644 --- a/br/pkg/task/restore_test.go +++ b/br/pkg/task/restore_test.go @@ -3,6 +3,7 @@ package task import ( + "context" "testing" "github.com/pingcap/tidb/br/pkg/restore" @@ -18,3 +19,24 @@ func TestRestoreConfigAdjust(t *testing.T) { require.Equal(t, restore.DefaultMergeRegionKeyCount, cfg.MergeSmallRegionKeyCount) require.Equal(t, restore.DefaultMergeRegionSizeBytes, cfg.MergeSmallRegionSizeBytes) } + +func TestconfigureRestoreClient(t *testing.T) { + cfg := Config{ + Concurrency: 1024, + } + restoreComCfg := RestoreCommonConfig{ + Online: true, + } + restoreCfg := &RestoreConfig{ + Config: cfg, + RestoreCommonConfig: restoreComCfg, + DdlBatchSize: 128, + } + client := &restore.Client{} + + ctx := context.Background() + err := configureRestoreClient(ctx, client, restoreCfg) + require.NoError(t, err) + require.Equal(t, client.GetBatchDdlSize(), 128) + require.True(t, true, client.IsOnline()) +} diff --git a/br/pkg/utils/backoff_test.go b/br/pkg/utils/backoff_test.go index 17aa3f72802e1..78f329708211b 100644 --- a/br/pkg/utils/backoff_test.go +++ b/br/pkg/utils/backoff_test.go @@ -136,7 +136,7 @@ func TestNewImportSSTBackofferWithSucess(t *testing.T) { } }, backoffer) require.Equal(t, 16, counter) - require.Nil(t, err) + require.NoError(t, err) } func TestNewDownloadSSTBackofferWithCancel(t *testing.T) { diff --git a/br/pkg/utils/pprof.go b/br/pkg/utils/pprof.go index efa25389b80d8..e11f2dcf0a5bc 100644 --- a/br/pkg/utils/pprof.go +++ b/br/pkg/utils/pprof.go @@ -11,7 +11,6 @@ import ( // #nosec // register HTTP handler for /debug/pprof "net/http" - _ "net/http/pprof" // nolint:gosec "github.com/pingcap/errors" "github.com/pingcap/failpoint" diff --git a/br/pkg/utils/safe_point.go b/br/pkg/utils/safe_point.go index 91f0335f8090b..512c489c42f45 100644 --- a/br/pkg/utils/safe_point.go +++ b/br/pkg/utils/safe_point.go @@ -11,8 +11,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" - "github.com/tikv/pd/pkg/tsoutil" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -37,7 +37,7 @@ func (sp BRServiceSafePoint) MarshalLogObject(encoder zapcore.ObjectEncoder) err encoder.AddString("ID", sp.ID) ttlDuration := time.Duration(sp.TTL) * time.Second encoder.AddString("TTL", ttlDuration.String()) - backupTime, _ := tsoutil.ParseTS(sp.BackupTS) + backupTime := oracle.GetTimeFromTS(sp.BackupTS) encoder.AddString("BackupTime", backupTime.String()) encoder.AddUint64("BackupTS", sp.BackupTS) return nil diff --git a/br/tests/br_incompatible_tidb_config/run.sh b/br/tests/br_incompatible_tidb_config/run.sh index 0034dd850249c..a6bdf089ce698 100755 --- a/br/tests/br_incompatible_tidb_config/run.sh +++ b/br/tests/br_incompatible_tidb_config/run.sh @@ -52,10 +52,17 @@ run_br --pd $PD_ADDR backup db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_ # restore run_sql "drop schema $DB;" +# restore with ddl(create table) job one by one +run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$TABLE" --ddl-batch-size=1 -run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$TABLE" +run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_TABLE" --ddl-batch-size=1 -run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_TABLE" +# restore +run_sql "drop schema $DB;" +# restore with batch create table +run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$TABLE" --ddl-batch-size=128 + +run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_TABLE" --ddl-batch-size=128 run_sql "drop schema $DB;" run_sql "create schema $DB;" diff --git a/br/tests/br_incremental_ddl/run.sh b/br/tests/br_incremental_ddl/run.sh index 68867a194a2a4..49b825498e2af 100755 --- a/br/tests/br_incremental_ddl/run.sh +++ b/br/tests/br_incremental_ddl/run.sh @@ -79,3 +79,26 @@ run_sql "INSERT INTO ${DB}.${TABLE}(c2) VALUES ('1');" run_sql "INSERT INTO ${DB}.${TABLE}_rename2(c) VALUES ('1');" run_sql "DROP DATABASE $DB;" + +# full restore with batch ddl +echo "full restore start..." +run_br restore table --db $DB --table $TABLE -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR --ddl-batch-size=128 +row_count_full=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') +# check full restore +if [ "${row_count_full}" != "${ROW_COUNT}" ];then + echo "TEST: [$TEST_NAME] full restore fail on database $DB" + exit 1 +fi +# incremental restore +echo "incremental restore start..." +run_br restore db --db $DB -s "local://$TEST_DIR/$DB/inc" --pd $PD_ADDR --ddl-batch-size=128 +row_count_inc=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') +# check full restore +if [ "${row_count_inc}" != "${ROW_COUNT}" ];then + echo "TEST: [$TEST_NAME] incremental restore fail on database $DB" + exit 1 +fi +run_sql "INSERT INTO ${DB}.${TABLE}(c2) VALUES ('1');" +run_sql "INSERT INTO ${DB}.${TABLE}_rename2(c) VALUES ('1');" + +run_sql "DROP DATABASE $DB;" \ No newline at end of file diff --git a/br/tests/lightning_incremental/data/incr.empty_table-schema.sql b/br/tests/lightning_incremental/data/incr.empty_table-schema.sql new file mode 100644 index 0000000000000..881156cb99fd5 --- /dev/null +++ b/br/tests/lightning_incremental/data/incr.empty_table-schema.sql @@ -0,0 +1 @@ +CREATE TABLE `empty_table` (id int primary key); diff --git a/br/tests/lightning_incremental/data1/incr.empty_table-schema.sql b/br/tests/lightning_incremental/data1/incr.empty_table-schema.sql new file mode 100644 index 0000000000000..881156cb99fd5 --- /dev/null +++ b/br/tests/lightning_incremental/data1/incr.empty_table-schema.sql @@ -0,0 +1 @@ +CREATE TABLE `empty_table` (id int primary key); diff --git a/br/tests/lightning_incremental/data1/incr.empty_table2-schema.sql b/br/tests/lightning_incremental/data1/incr.empty_table2-schema.sql new file mode 100644 index 0000000000000..610412d940815 --- /dev/null +++ b/br/tests/lightning_incremental/data1/incr.empty_table2-schema.sql @@ -0,0 +1 @@ +CREATE TABLE `empty_table2` (id int primary key, s varchar(16)); diff --git a/br/tests/lightning_incremental/run.sh b/br/tests/lightning_incremental/run.sh index f04630055936a..4cdd5a53ec74b 100644 --- a/br/tests/lightning_incremental/run.sh +++ b/br/tests/lightning_incremental/run.sh @@ -18,60 +18,66 @@ set -eu check_cluster_version 4 0 0 "incremental restore" || exit 0 -DB_NAME=incr +run_lightning_and_check_meta() { + run_lightning --backend local "$@" + # check metadata table is not exist + run_sql "SHOW DATABASES like 'lightning_metadata';" + check_not_contains "Database: lightning_metadata" +} -for backend in importer local; do - run_sql "DROP DATABASE IF EXISTS incr;" - run_lightning --backend $backend +DB_NAME=incr - for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do - run_sql "SELECT count(*) from incr.$tbl" - check_contains "count(*): 3" - done +run_sql "DROP DATABASE IF EXISTS incr;" +run_sql "DROP DATABASE IF EXISTS lightning_metadata;" +run_lightning_and_check_meta - for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do - if [ "$tbl" = "auto_random" ]; then - run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM incr.$tbl" - else - run_sql "SELECT id as inc FROM incr.$tbl" - fi - check_contains 'inc: 1' - check_contains 'inc: 2' - check_contains 'inc: 3' - done +for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do + run_sql "SELECT count(*) from incr.$tbl" + check_contains "count(*): 3" +done - for tbl in pk_auto_inc rowid_uk_inc; do - run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" - check_contains "group_concat(v): a,b,c" - done +for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do + if [ "$tbl" = "auto_random" ]; then + run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM incr.$tbl" + else + run_sql "SELECT id as inc FROM incr.$tbl" + fi + check_contains 'inc: 1' + check_contains 'inc: 2' + check_contains 'inc: 3' +done - run_sql "SELECT sum(pk) from incr.uk_auto_inc;" - check_contains "sum(pk): 6" +for tbl in pk_auto_inc rowid_uk_inc; do + run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" + check_contains "group_concat(v): a,b,c" +done - # incrementally import all data in data1 - run_lightning --backend $backend -d "tests/$TEST_NAME/data1" +run_sql "SELECT sum(pk) from incr.uk_auto_inc;" +check_contains "sum(pk): 6" - for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do - run_sql "SELECT count(*) from incr.$tbl" - check_contains "count(*): 6" - done +# incrementally import all data in data1 +run_lightning_and_check_meta -d "tests/$TEST_NAME/data1" - for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do - if [ "$tbl" = "auto_random" ]; then - run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM incr.$tbl" - else - run_sql "SELECT id as inc FROM incr.$tbl" - fi - check_contains 'inc: 4' - check_contains 'inc: 5' - check_contains 'inc: 6' - done +for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do + run_sql "SELECT count(*) from incr.$tbl" + check_contains "count(*): 6" +done - for tbl in pk_auto_inc rowid_uk_inc; do - run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" - check_contains "group_concat(v): a,b,c,d,e,f" - done +for tbl in auto_random pk_auto_inc rowid_uk_inc uk_auto_inc; do + if [ "$tbl" = "auto_random" ]; then + run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM incr.$tbl" + else + run_sql "SELECT id as inc FROM incr.$tbl" + fi + check_contains 'inc: 4' + check_contains 'inc: 5' + check_contains 'inc: 6' +done - run_sql "SELECT sum(pk) from incr.uk_auto_inc;" - check_contains "sum(pk): 21" +for tbl in pk_auto_inc rowid_uk_inc; do + run_sql "SELECT group_concat(v) from incr.$tbl group by 'all';" + check_contains "group_concat(v): a,b,c,d,e,f" done + +run_sql "SELECT sum(pk) from incr.uk_auto_inc;" +check_contains "sum(pk): 21" diff --git a/cmd/ddltest/index_test.go b/cmd/ddltest/index_test.go index 28cba8e03ee5f..4bbbadd292b88 100644 --- a/cmd/ddltest/index_test.go +++ b/cmd/ddltest/index_test.go @@ -16,20 +16,15 @@ package ddltest import ( "fmt" - "io" "math" "sync" "sync/atomic" "testing" "time" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/table/tables" - "github.com/pingcap/tidb/types" "github.com/stretchr/testify/require" goctx "golang.org/x/net/context" ) @@ -44,90 +39,12 @@ func getIndex(t table.Table, name string) table.Index { return nil } -func (s *ddlSuite) checkAddIndex(t *testing.T, indexInfo *model.IndexInfo) { - ctx := s.ctx - err := ctx.NewTxn(goctx.Background()) - require.NoError(t, err) - tbl := s.getTable(t, "test_index") - - // read handles form table - handles := kv.NewHandleMap() - err = tables.IterRecords(tbl, ctx, tbl.Cols(), - func(h kv.Handle, data []types.Datum, cols []*table.Column) (bool, error) { - handles.Set(h, struct{}{}) - return true, nil - }) - require.NoError(t, err) - - // read handles from index - idx := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), indexInfo) - err = ctx.NewTxn(goctx.Background()) - require.NoError(t, err) - txn, err := ctx.Txn(false) - require.NoError(t, err) - defer func() { - err = txn.Rollback() - require.NoError(t, err) - }() - - it, err := idx.SeekFirst(txn) - require.NoError(t, err) - defer it.Close() - - for { - _, h, err := it.Next() - if terror.ErrorEqual(err, io.EOF) { - break - } - - require.NoError(t, err) - _, ok := handles.Get(h) - require.True(t, ok) - handles.Delete(h) - } - - require.Equal(t, 0, handles.Len()) -} - func (s *ddlSuite) checkDropIndex(t *testing.T, indexInfo *model.IndexInfo) { gcWorker, err := gcworker.NewMockGCWorker(s.store) require.NoError(t, err) err = gcWorker.DeleteRanges(goctx.Background(), uint64(math.MaxInt32)) require.NoError(t, err) - - ctx := s.ctx - err = ctx.NewTxn(goctx.Background()) - require.NoError(t, err) - tbl := s.getTable(t, "test_index") - - // read handles from index - idx := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), indexInfo) - err = ctx.NewTxn(goctx.Background()) - require.NoError(t, err) - txn, err := ctx.Txn(false) - require.NoError(t, err) - defer func() { - err := txn.Rollback() - require.NoError(t, err) - }() - - it, err := idx.SeekFirst(txn) - require.NoError(t, err) - defer it.Close() - - handles := kv.NewHandleMap() - for { - _, h, err := it.Next() - if terror.ErrorEqual(err, io.EOF) { - break - } - - require.NoError(t, err) - handles.Set(h, struct{}{}) - } - - // TODO: Uncomment this after apply pool is finished - // c.Assert(handles.Len(), Equals, 0) + s.mustExec(fmt.Sprintf("admin check table %s", indexInfo.Table.String())) } // TestIndex operations on table test_index (c int, c1 bigint, c2 double, c3 varchar(256), primary key(c)). @@ -194,7 +111,7 @@ func TestIndex(t *testing.T) { if col.Add { require.NotNil(t, index) oldIndex = index - s.checkAddIndex(t, index.Meta()) + s.mustExec("admin check table test_index") } else { require.Nil(t, index) s.checkDropIndex(t, oldIndex.Meta()) diff --git a/cmd/ddltest/main_test.go b/cmd/ddltest/main_test.go index 810ba80b9a974..e4c2a6834450a 100644 --- a/cmd/ddltest/main_test.go +++ b/cmd/ddltest/main_test.go @@ -33,7 +33,7 @@ func TestMain(m *testing.M) { os.Exit(1) } opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), diff --git a/cmd/ddltest/random.go b/cmd/ddltest/random_test.go similarity index 100% rename from cmd/ddltest/random.go rename to cmd/ddltest/random_test.go diff --git a/cmd/explaintest/disable_new_collation.toml b/cmd/explaintest/disable_new_collation.toml index 138892acf40db..c66f92d0b30c8 100644 --- a/cmd/explaintest/disable_new_collation.toml +++ b/cmd/explaintest/disable_new_collation.toml @@ -15,6 +15,7 @@ lease = "0" mem-quota-query = 34359738368 host = "127.0.0.1" +new_collations_enabled_on_first_bootstrap = false [status] status-host = "127.0.0.1" diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index 8b6ad2a86b293..aa644842c3795 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -227,6 +227,14 @@ func (t *tester) parserErrorHandle(query query, err error) error { err = nil break } + if strings.Contains(err.Error(), expectedErr) { + if t.enableQueryLog { + t.buf.WriteString(fmt.Sprintf("%s\n", query.Query)) + } + t.buf.WriteString(fmt.Sprintf("%s\n", err)) + err = nil + break + } } if err != nil { @@ -353,12 +361,14 @@ func (t *tester) execute(query query) error { } if err != nil && len(t.expectedErrs) > 0 { - // TODO: check whether this err is expected. - // but now we think it is. - - // output expected err - t.buf.WriteString(fmt.Sprintf("%s\n", err)) - err = nil + for _, expectErr := range t.expectedErrs { + if strings.Contains(err.Error(), expectErr) { + // output expected err + t.buf.WriteString(fmt.Sprintf("%s\n", err)) + err = nil + break + } + } } // clear expected errors after we execute the first query t.expectedErrs = nil diff --git a/cmd/explaintest/r/collation_misc_disabled.result b/cmd/explaintest/r/collation_misc_disabled.result new file mode 100644 index 0000000000000..33f0ebe26f701 --- /dev/null +++ b/cmd/explaintest/r/collation_misc_disabled.result @@ -0,0 +1,96 @@ +create database collation_misc; +use collation_misc; +create table t1(a varchar(20) charset utf8); +insert into t1 values ("t1_value"); +alter table t1 collate uTf8mB4_uNiCoDe_Ci charset Utf8mB4 charset uTF8Mb4 collate UTF8MB4_BiN; +alter table t1 modify column a varchar(20) charset utf8mb4; +select * from t1; +a +t1_value +create table t(a varchar(20) charset latin1); +insert into t values ("t_value"); +alter table t modify column a varchar(20) charset latin1; +select * from t; +a +t_value +alter table t modify column a varchar(20) charset utf8; +Error 8200: Unsupported modify charset from latin1 to utf8 +alter table t modify column a varchar(20) charset utf8mb4; +Error 8200: Unsupported modify charset from latin1 to utf8mb4 +alter table t modify column a varchar(20) charset utf8 collate utf8_bin; +Error 8200: Unsupported modify charset from latin1 to utf8 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8mb4_general_ci; +Error 8200: Unsupported modify charset from latin1 to utf8mb4 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8bin; +[ddl:1273]Unknown collation: 'utf8bin' +alter table t collate LATIN1_GENERAL_CI charset utf8 collate utf8_bin; +Error 1302: Conflicting declarations: 'CHARACTER SET latin1' and 'CHARACTER SET utf8' +alter table t collate LATIN1_GENERAL_CI collate UTF8MB4_UNICODE_ci collate utf8_bin; +Error 1253: COLLATION 'utf8mb4_unicode_ci' is not valid for CHARACTER SET 'latin1' +create database if not exists cd_test_utf8 CHARACTER SET utf8 COLLATE utf8_bin; +create database if not exists cd_test_latin1 CHARACTER SET latin1 COLLATE latin1_swedish_ci; +use cd_test_utf8; +select @@character_set_database; +@@character_set_database +utf8 +select @@collation_database; +@@collation_database +utf8_bin +use cd_test_latin1; +select @@character_set_database; +@@character_set_database +latin1 +select @@collation_database; +@@collation_database +latin1_swedish_ci +create database if not exists test_db CHARACTER SET latin1 COLLATE latin1_swedish_ci; +with cte as (select cast('2010-09-09' as date) a union select '2010-09-09 ') select count(*) from cte; +count(*) +2 +set names utf8mb4 collate utf8mb4_general_ci; +select position('a' in 'AA'); +position('a' in 'AA') +0 +select locate('a', 'AA'); +locate('a', 'AA') +0 +select locate('a', 'a'); +locate('a', 'a') +1 +set names utf8mb4; +SELECT default_collate_name, maxlen FROM information_schema.character_sets ORDER BY character_set_name; +default_collate_name maxlen +ascii_bin 1 +binary 1 +gbk_bin 2 +latin1_bin 1 +utf8_bin 3 +utf8mb4_bin 4 +SELECT character_set_name, id, sortlen FROM information_schema.collations ORDER BY collation_name, id; +character_set_name id sortlen +ascii 65 1 +binary 63 1 +gbk 87 1 +latin1 47 1 +utf8 83 1 +utf8mb4 46 1 +select * from information_schema.COLLATION_CHARACTER_SET_APPLICABILITY where COLLATION_NAME='utf8mb4_bin'; +COLLATION_NAME CHARACTER_SET_NAME +utf8mb4_bin utf8mb4 +show charset; +Charset Description Default collation Maxlen +ascii US ASCII ascii_bin 1 +binary binary binary 1 +gbk Chinese Internal Code Specification gbk_bin 2 +latin1 Latin1 latin1_bin 1 +utf8 UTF-8 Unicode utf8_bin 3 +utf8mb4 UTF-8 Unicode utf8mb4_bin 4 +show collation; +Collation Charset Id Default Compiled Sortlen +utf8mb4_bin utf8mb4 46 Yes Yes 1 +latin1_bin latin1 47 Yes Yes 1 +binary binary 63 Yes Yes 1 +ascii_bin ascii 65 Yes Yes 1 +utf8_bin utf8 83 Yes Yes 1 +gbk_bin gbk 87 Yes Yes 1 +use test; diff --git a/cmd/explaintest/r/collation_misc_enabled.result b/cmd/explaintest/r/collation_misc_enabled.result new file mode 100644 index 0000000000000..8f75d4e18d151 --- /dev/null +++ b/cmd/explaintest/r/collation_misc_enabled.result @@ -0,0 +1,109 @@ +create database collation_misc; +use collation_misc; +create table t1(a varchar(20) charset utf8); +insert into t1 values ("t1_value"); +alter table t1 collate uTf8mB4_uNiCoDe_Ci charset Utf8mB4 charset uTF8Mb4 collate UTF8MB4_BiN; +alter table t1 modify column a varchar(20) charset utf8mb4; +select * from t1; +a +t1_value +create table t(a varchar(20) charset latin1); +insert into t values ("t_value"); +alter table t modify column a varchar(20) charset latin1; +select * from t; +a +t_value +alter table t modify column a varchar(20) charset utf8; +Error 8200: Unsupported modify charset from latin1 to utf8 +alter table t modify column a varchar(20) charset utf8mb4; +Error 8200: Unsupported modify charset from latin1 to utf8mb4 +alter table t modify column a varchar(20) charset utf8 collate utf8_bin; +Error 8200: Unsupported modify charset from latin1 to utf8 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8mb4_general_ci; +Error 8200: Unsupported modify charset from latin1 to utf8mb4 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8bin; +[ddl:1273]Unknown collation: 'utf8bin' +alter table t collate LATIN1_GENERAL_CI charset utf8 collate utf8_bin; +Error 1273: Unsupported collation when new collation is enabled: 'latin1_general_ci' +alter table t collate LATIN1_GENERAL_CI collate UTF8MB4_UNICODE_ci collate utf8_bin; +Error 1273: Unsupported collation when new collation is enabled: 'latin1_general_ci' +create database if not exists cd_test_utf8 CHARACTER SET utf8 COLLATE utf8_bin; +create database if not exists cd_test_latin1 CHARACTER SET latin1 COLLATE latin1_swedish_ci; +Error 1273: Unsupported collation when new collation is enabled: 'latin1_swedish_ci' +use cd_test_utf8; +select @@character_set_database; +@@character_set_database +utf8 +select @@collation_database; +@@collation_database +utf8_bin +use cd_test_latin1; +Error 1049: Unknown database 'cd_test_latin1' +select @@character_set_database; +@@character_set_database +utf8 +select @@collation_database; +@@collation_database +utf8_bin +create database if not exists test_db CHARACTER SET latin1 COLLATE latin1_swedish_ci; +Error 1273: Unsupported collation when new collation is enabled: 'latin1_swedish_ci' +with cte as (select cast('2010-09-09' as date) a union select '2010-09-09 ') select count(*) from cte; +count(*) +1 +set names utf8mb4 collate utf8mb4_general_ci; +select position('a' in 'AA'); +position('a' in 'AA') +1 +select locate('a', 'AA'); +locate('a', 'AA') +1 +select locate('a', 'a'); +locate('a', 'a') +1 +set names utf8mb4; +SELECT default_collate_name, maxlen FROM information_schema.character_sets ORDER BY character_set_name; +default_collate_name maxlen +ascii_bin 1 +binary 1 +gbk_chinese_ci 2 +latin1_bin 1 +utf8_bin 3 +utf8mb4_bin 4 +SELECT character_set_name, id, sortlen FROM information_schema.collations ORDER BY collation_name, id; +character_set_name id sortlen +ascii 65 1 +binary 63 1 +gbk 87 1 +gbk 28 1 +latin1 47 1 +utf8 83 1 +utf8 33 1 +utf8 192 1 +utf8mb4 46 1 +utf8mb4 45 1 +utf8mb4 224 1 +select * from information_schema.COLLATION_CHARACTER_SET_APPLICABILITY where COLLATION_NAME='utf8mb4_bin'; +COLLATION_NAME CHARACTER_SET_NAME +utf8mb4_bin utf8mb4 +show charset; +Charset Description Default collation Maxlen +ascii US ASCII ascii_bin 1 +binary binary binary 1 +gbk Chinese Internal Code Specification gbk_chinese_ci 2 +latin1 Latin1 latin1_bin 1 +utf8 UTF-8 Unicode utf8_bin 3 +utf8mb4 UTF-8 Unicode utf8mb4_bin 4 +show collation; +Collation Charset Id Default Compiled Sortlen +ascii_bin ascii 65 Yes Yes 1 +binary binary 63 Yes Yes 1 +gbk_bin gbk 87 Yes 1 +gbk_chinese_ci gbk 28 Yes Yes 1 +latin1_bin latin1 47 Yes Yes 1 +utf8_bin utf8 83 Yes Yes 1 +utf8_general_ci utf8 33 Yes 1 +utf8_unicode_ci utf8 192 Yes 1 +utf8mb4_bin utf8mb4 46 Yes Yes 1 +utf8mb4_general_ci utf8mb4 45 Yes 1 +utf8mb4_unicode_ci utf8mb4 224 Yes 1 +use test; diff --git a/cmd/explaintest/r/collation_pointget_disabled.result b/cmd/explaintest/r/collation_pointget_disabled.result new file mode 100644 index 0000000000000..96b2a7aa58ca5 --- /dev/null +++ b/cmd/explaintest/r/collation_pointget_disabled.result @@ -0,0 +1,192 @@ +create database collation_point_get; +use collation_point_get; +drop table if exists t; +create table t(a char(2), b char(2), index idx_1(a)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +select * from t tmp where a = "aa"; +a b +aa bb +select * from t tmp where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +select * from t where a = "a "; +a b +drop table if exists t; +create table t(a char(2) binary, b char(2), index idx_1(a)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +select * from t where a = "a "; +a b +select * from t where a = ""; +a b + +select * from t where a = " "; +a b +select * from t where a = " "; +a b +select * from t where a = " "; +a b +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select * from t tmp where a = "aa"; +a b +aa bb +select * from t tmp where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +a b +a b +select * from t tmp where a = "a "; +a b +select * from t tmp where a = "a "; +a b +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +a b +a b +select * from t tmp where a = "a "; +a b +select * from t tmp where a = "a "; +a b +select * from t tmp where a = ""; +a b + +select * from t tmp where a = " "; +a b +select * from t tmp where a = " "; +a b +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select *, a from t tmp where a = "aa"; +a b a +aa bb aa +select tmp.* from t tmp where a = "aa"; +a b +aa bb +select tmp.a, tmp.b from t tmp where a = "aa"; +a b +aa bb +select tmp.*, tmp.a, tmp.b from t tmp where a = "aa"; +a b a b +aa bb aa bb +select tmp.* from t tmp where a = "aab"; +a b +select tmp.a, tmp.b from t tmp where a = "aab"; +a b +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +a b a b +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +a b a b +select * from t tmp where tmp.a = "aa"; +a b +aa bb +select a, b from t tmp where tmp.a = "aa"; +a b +aa bb +select *, a, b from t tmp where tmp.a = "aa"; +a b a b +aa bb aa bb +select a from t where xxxxx.a = "aa"; +Error 1054: Unknown column 'xxxxx.a' in 'where clause' +select xxxxx.a from t where a = "aa"; +Error 1054: Unknown column 'xxxxx.a' in 'field list' +select a from t tmp where t.a = "aa"; +Error 1054: Unknown column 't.a' in 'where clause' +select t.a from t tmp where a = "aa"; +Error 1054: Unknown column 't.a' in 'field list' +select t.* from t tmp where a = "aa"; +Error 1051: Unknown table 't' +drop table if exists t; +create table t(a char(4) primary key, b char(4)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +select * from t where a = "a "; +a b +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +select * from t where a = "a "; +a b +select * from t where a = ""; +a b + +select * from t where a = " "; +a b +select * from t where a = " "; +a b +drop table if exists t; +create table t(a varchar(2) primary key, b varchar(2)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +drop table if exists t; +create table t(a varchar(2) binary primary key, b varchar(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +select * from t where a = ""; +a b +select * from t where a = " "; +a b + +select * from t where a = " "; +a b +use mysql; diff --git a/cmd/explaintest/r/collation_pointget_enabled.result b/cmd/explaintest/r/collation_pointget_enabled.result new file mode 100644 index 0000000000000..c5bcd58c15ba2 --- /dev/null +++ b/cmd/explaintest/r/collation_pointget_enabled.result @@ -0,0 +1,217 @@ +create database collation_point_get; +use collation_point_get; +drop table if exists t; +create table t(a char(2), b char(2), index idx_1(a)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +select * from t tmp where a = "aa"; +a b +aa bb +select * from t tmp where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +drop table if exists t; +create table t(a char(2) binary, b char(2), index idx_1(a)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = ""; +a b + +select * from t where a = " "; +a b + +select * from t where a = " "; +a b + +select * from t where a = " "; +a b + +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select * from t tmp where a = "aa"; +a b +aa bb +select * from t tmp where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +a b +a b +select * from t tmp where a = "a "; +a b +a b +select * from t tmp where a = "a "; +a b +a b +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +a b +a b +select * from t tmp where a = "a "; +a b +a b +select * from t tmp where a = "a "; +a b +a b +select * from t tmp where a = ""; +a b + +select * from t tmp where a = " "; +a b + +select * from t tmp where a = " "; +a b + +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select *, a from t tmp where a = "aa"; +a b a +aa bb aa +select tmp.* from t tmp where a = "aa"; +a b +aa bb +select tmp.a, tmp.b from t tmp where a = "aa"; +a b +aa bb +select tmp.*, tmp.a, tmp.b from t tmp where a = "aa"; +a b a b +aa bb aa bb +select tmp.* from t tmp where a = "aab"; +a b +select tmp.a, tmp.b from t tmp where a = "aab"; +a b +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +a b a b +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +a b a b +select * from t tmp where tmp.a = "aa"; +a b +aa bb +select a, b from t tmp where tmp.a = "aa"; +a b +aa bb +select *, a, b from t tmp where tmp.a = "aa"; +a b a b +aa bb aa bb +select a from t where xxxxx.a = "aa"; +Error 1054: Unknown column 'xxxxx.a' in 'where clause' +select xxxxx.a from t where a = "aa"; +Error 1054: Unknown column 'xxxxx.a' in 'field list' +select a from t tmp where t.a = "aa"; +Error 1054: Unknown column 't.a' in 'where clause' +select t.a from t tmp where a = "aa"; +Error 1054: Unknown column 't.a' in 'field list' +select t.* from t tmp where a = "aa"; +Error 1051: Unknown table 't' +drop table if exists t; +create table t(a char(4) primary key, b char(4)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = ""; +a b + +select * from t where a = " "; +a b + +select * from t where a = " "; +a b + +drop table if exists t; +create table t(a varchar(2) primary key, b varchar(2)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +a b +aa bb +select * from t where a = "aab"; +a b +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +drop table if exists t; +create table t(a varchar(2) binary primary key, b varchar(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = "a "; +a b +a b +select * from t where a = ""; +a b + +select * from t where a = " "; +a b + +select * from t where a = " "; +a b + +use mysql; diff --git a/cmd/explaintest/r/common_collation.result b/cmd/explaintest/r/common_collation.result index 53a8250653ea3..235ce7fce3d0d 100644 --- a/cmd/explaintest/r/common_collation.result +++ b/cmd/explaintest/r/common_collation.result @@ -13,6 +13,21 @@ coercibility(convert('a' using utf8mb4)) select coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci); coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci) 0 +drop table if exists t; +create table t (a char(20), b blob(100), c text, d json, e timestamp, f set('a一','b二','c三','d四'), g text, h enum('a一','b二','c三','d四') default 'c三'); +insert into t values ('你好', '你好', '你好', '{\"测试\": \"你好\"}', '2018-10-13', 1, '你好', 'a一'); +select coercibility(a), coercibility(b), coercibility(c), coercibility(d), coercibility(e), coercibility(f), coercibility(g), coercibility(h) from t; +coercibility(a) coercibility(b) coercibility(c) coercibility(d) coercibility(e) coercibility(f) coercibility(g) coercibility(h) +2 2 2 2 5 2 2 2 +select collation(d), collation(upper(d)), collation(elt(1, d, 0x12)), collation(elt(1, elt(1, d, 0x12), 0x12)), collation(elt(1, d, b)) from t; +collation(d) collation(upper(d)) collation(elt(1, d, 0x12)) collation(elt(1, elt(1, d, 0x12), 0x12)) collation(elt(1, d, b)) +binary utf8mb4_bin utf8mb4_bin utf8mb4_bin binary +drop table t; +create table t(a binary, b json, c char charset gbk); +insert into t values ('a', '{"a":"b"}', 'a'); +select collation(concat(a, b)), collation(concat(b, a)), collation(concat(0x61, b)), collation(concat(b, 0x61)), collation(concat(c, b)), collation(concat(b, c)) from t; +collation(concat(a, b)) collation(concat(b, a)) collation(concat(0x61, b)) collation(concat(b, 0x61)) collation(concat(c, b)) collation(concat(b, c)) +binary binary utf8mb4_bin utf8mb4_bin utf8mb4_bin utf8mb4_bin DROP TABLE IF EXISTS t2; CREATE TABLE t2 ( id INT NOT NULL PRIMARY KEY auto_increment, diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index fb209b6bcabd5..8272cd691a007 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -523,7 +523,7 @@ a b select @@tidb_allow_function_for_expression_index; @@tidb_allow_function_for_expression_index -lower, md5, reverse, upper, vitess_hash +lower, md5, reverse, tidb_shard, upper, vitess_hash CREATE TABLE `PK_S_MULTI_30_tmp` ( `COL1` double NOT NULL, `COL2` double NOT NULL, @@ -577,4 +577,43 @@ replace into t (c_int, c_str) values (9, "beautiful hermann"); select reverse(c_str) from t use index(idx); reverse(c_str) nnamreh lufituaeb +drop table if exists t1; +drop table if exists t2; +create table t1 (c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci, c_datetime datetime, c_timestamp timestamp, c_double double, c_decimal decimal(12, 6), c_enum enum('blue','green','red','yellow','white','orange','purple'), primary key (c_datetime) , key(c_int) , key(c_datetime) , key((c_int + 1)), key((c_int -1)), key((lower(c_str))), key((md5(c_str))), key((reverse(c_str))), key((upper(c_str)))); +create table t2 like t1; +insert into t1 values(11, 'loving grothendieck', '2020-02-02 19:25:49', '2020-03-27 15:17:14', 3.269, 1.851000, 'white' ); +insert into t1 values(11, 'quirky kapitsa' , '2020-06-21 03:55:31', '2020-02-29 17:02:48', 6.94, 1.851000, 'yellow'); +insert into t1 values( 7, 'boring bouman' , '2020-05-10 00:01:04', '2020-02-01 20:18:00', 84.096168, 6.996000, 'white' ); +insert into t2 values( 11, 'wizardly antonelli', '2020-01-30 17:27:17', '2020-01-01 10:05:31', 6.886177, 6.332000, 'green' ); +insert into t2 values( 2, 'angry kapitsa' , '2020-03-30 05:09:44', '2020-02-15 00:36:52', 95.798378, 3.118000, 'blue' ); +insert into t2 values( 7, 'dreamy shamir' , '2020-05-28 14:13:42', '2020-06-02 07:23:22', 26.623227, 3.105000, 'orange'); +begin; +delete from t2 where c_decimal > c_double/2 order by c_int, c_str, c_double, c_decimal limit 1; +desc format='brief' select t2.c_enum from t2,t1 where t1.c_int - 1 = t2.c_int - 1 order by t2.c_enum; +id estRows task access object operator info +Sort 12487.50 root test.t2.c_enum +└─HashJoin 12487.50 root inner join, equal:[eq(minus(test.t1.c_int, 1), minus(test.t2.c_int, 1))] + ├─IndexReader(Build) 9990.00 root index:IndexFullScan + │ └─IndexFullScan 9990.00 cop[tikv] table:t1, index:expression_index_2(`c_int` - 1) keep order:false, stats:pseudo + └─Projection(Probe) 10000.00 root test.t2.c_enum, minus(test.t2.c_int, 1), test.t2._tidb_rowid + └─UnionScan 8000.00 root not(isnull(minus(test.t2.c_int, 1))) + └─Selection 8000.00 root not(isnull(minus(test.t2.c_int, 1))) + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo +select t2.c_enum from t2,t1 where t1.c_int - 1 = t2.c_int - 1 order by t2.c_enum; +c_enum +orange +drop table t1, t2; drop table t; +drop table if exists t1,t2; +create table t1 (c_int int, c_str varchar(40) ,primary key (c_int) , key(c_str(36)) , key((c_int + 1))) partition by hash (c_int) partitions 4 ; +create table t2 like t1 ; +insert into t1 values (1, 'sleepy kowalevski'); +insert into t2 values (3, 'unruffled chaplygin'); +select (select t2.c_str from t2 where t2.c_int + 1 = 4 order by t2.c_str) x from t1; +x +unruffled chaplygin +select (select t2.c_str from t2 where t2.c_int = 3 order by t2.c_str) x from t1; +x +unruffled chaplygin +drop table t1,t2; diff --git a/cmd/explaintest/r/explain_shard_index.result b/cmd/explaintest/r/explain_shard_index.result new file mode 100644 index 0000000000000..79276d331286c --- /dev/null +++ b/cmd/explaintest/r/explain_shard_index.result @@ -0,0 +1,73 @@ +use test; +drop table if exists test3, test5; +create table test3(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a)); +create table test5(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a,b)); +explain format=brief select * from test3 where a=100; +id estRows task access object operator info +Projection 1.00 root test.test3.id, test.test3.a, test.test3.b +└─Point_Get 1.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) +explain format=brief select * from test3 where a=100 and (b = 100 or b = 200); +id estRows task access object operator info +Projection 0.00 root test.test3.id, test.test3.a, test.test3.b +└─Selection 0.00 root or(eq(test.test3.b, 100), eq(test.test3.b, 200)) + └─Point_Get 1.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) +explain format=brief select * from test3 where tidb_shard(a) = 8; +id estRows task access object operator info +Projection 10.00 root test.test3.id, test.test3.a, test.test3.b +└─IndexLookUp 10.00 root + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:test3, index:uk_expr(tidb_shard(`a`), a) range:[8,8], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 10.00 cop[tikv] table:test3 keep order:false, stats:pseudo +explain format=brief select * from test3 where a=100 or b = 200; +id estRows task access object operator info +Projection 8000.00 root test.test3.id, test.test3.a, test.test3.b +└─Selection 8000.00 root or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), eq(test.test3.b, 200)) + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:test3 keep order:false, stats:pseudo +explain format=brief select * from test3 where a=100 or a = 300; +id estRows task access object operator info +Projection 2.00 root test.test3.id, test.test3.a, test.test3.b +└─Batch_Point_Get 2.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) keep order:false, desc:false +explain format=brief select * from test3 where a=100 or a = 300 or a > 997; +id estRows task access object operator info +Projection 8000.00 root test.test3.id, test.test3.a, test.test3.b +└─Selection 8000.00 root or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), or(and(eq(tidb_shard(test.test3.a), 227), eq(test.test3.a, 300)), gt(test.test3.a, 997))) + └─TableReader 10000.00 root data:TableFullScan + └─TableFullScan 10000.00 cop[tikv] table:test3 keep order:false, stats:pseudo +explain format=brief select * from test3 where ((a=100 and b = 100) or a = 200) and b = 300; +id estRows task access object operator info +Projection 0.01 root test.test3.id, test.test3.a, test.test3.b +└─TableReader 0.01 root data:Selection + └─Selection 0.01 cop[tikv] eq(test.test3.b, 300), or(0, eq(test.test3.a, 200)) + └─TableFullScan 10000.00 cop[tikv] table:test3 keep order:false, stats:pseudo +explain format=brief select * from test3 where a = b; +id estRows task access object operator info +Projection 8000.00 root test.test3.id, test.test3.a, test.test3.b +└─TableReader 8000.00 root data:Selection + └─Selection 8000.00 cop[tikv] eq(test.test3.a, test.test3.b) + └─TableFullScan 10000.00 cop[tikv] table:test3 keep order:false, stats:pseudo +explain format=brief select * from test3 where a = b and b = 100; +id estRows task access object operator info +Projection 0.00 root test.test3.id, test.test3.a, test.test3.b +└─Selection 0.00 root eq(test.test3.b, 100) + └─Point_Get 1.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) +explain format=brief select * from test5 where a=100 and b = 100; +id estRows task access object operator info +Projection 1.00 root test.test5.id, test.test5.a, test.test5.b +└─Point_Get 1.00 root table:test5, index:uk_expr(tidb_shard(`a`), a, b) +explain format=brief select * from test5 where (a=100 and b = 100) or (a=200 and b = 200); +id estRows task access object operator info +Projection 2.00 root test.test5.id, test.test5.a, test.test5.b +└─Batch_Point_Get 2.00 root table:test5, index:uk_expr(tidb_shard(`a`), a, b) keep order:false, desc:false +explain format=brief select a+b from test5 where (a, b) in ((100, 100), (200, 200)); +id estRows task access object operator info +Projection 2.00 root plus(test.test5.a, test.test5.b)->Column#5 +└─Batch_Point_Get 2.00 root table:test5, index:uk_expr(tidb_shard(`a`), a, b) keep order:false, desc:false +explain format=brief SELECT * FROM test3 WHERE a IN (100); +id estRows task access object operator info +Projection 1.00 root test.test3.id, test.test3.a, test.test3.b +└─Point_Get 1.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) +explain format=brief SELECT * FROM test3 WHERE a IN (100, 200, 300); +id estRows task access object operator info +Projection 3.00 root test.test3.id, test.test3.a, test.test3.b +└─Batch_Point_Get 3.00 root table:test3, index:uk_expr(tidb_shard(`a`), a) keep order:false, desc:false +drop table if exists test3, test5; diff --git a/cmd/explaintest/r/generated_columns.result b/cmd/explaintest/r/generated_columns.result index 970f00880ac5d..fa5d4f79b5906 100644 --- a/cmd/explaintest/r/generated_columns.result +++ b/cmd/explaintest/r/generated_columns.result @@ -170,9 +170,10 @@ INSERT INTO person (name, address_info) VALUES ("John", CAST('{"city_no": 1}' AS EXPLAIN format = 'brief' SELECT name FROM person where city_no=1; id estRows task access object operator info Projection 10.00 root test.person.name -└─IndexLookUp 10.00 root - ├─IndexRangeScan(Build) 10.00 cop[tikv] table:person, index:city_no(city_no) range:[1,1], keep order:false, stats:pseudo - └─TableRowIDScan(Probe) 10.00 cop[tikv] table:person keep order:false, stats:pseudo +└─Projection 10.00 root test.person.name, test.person.city_no + └─IndexLookUp 10.00 root + ├─IndexRangeScan(Build) 10.00 cop[tikv] table:person, index:city_no(city_no) range:[1,1], keep order:false, stats:pseudo + └─TableRowIDScan(Probe) 10.00 cop[tikv] table:person keep order:false, stats:pseudo DROP TABLE IF EXISTS t1; CREATE TABLE t1 (a INT, b INT GENERATED ALWAYS AS (-a) VIRTUAL, diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 014aac58825b5..48080df7d23f2 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -705,11 +705,11 @@ c1 c2 c3 c4 c5 explain select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and truncate(c1, 1) = 1 order by 1; id estRows task access object operator info Sort_5 4433.77 root test.t1.c1 -└─Selection_8 4433.77 root eq(truncate(test.t1.c1, 1), 1) - └─IndexMerge_12 5542.21 root - ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo - ├─IndexRangeScan_10(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo - └─TableRowIDScan_11(Probe) 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo +└─IndexMerge_12 4433.77 root + ├─IndexRangeScan_8(Build) 3323.33 cop[tikv] table:t1, index:c1(c1) range:[-inf,10), keep order:false, stats:pseudo + ├─IndexRangeScan_9(Build) 3323.33 cop[tikv] table:t1, index:c2(c2) range:[-inf,10), keep order:false, stats:pseudo + └─Selection_11(Probe) 4433.77 cop[tikv] eq(truncate(test.t1.c1, 1), 1) + └─TableRowIDScan_10 5542.21 cop[tikv] table:t1 keep order:false, stats:pseudo select /*+ use_index_merge(t1) */ * from t1 where (c1 < 10 or c2 < 10) and truncate(c1, 1) = 1 order by 1; c1 c2 c3 c4 c5 1 1 1 1 1 diff --git a/cmd/explaintest/r/new_character_set.result b/cmd/explaintest/r/new_character_set.result index db70e0d82f72b..18b0f44662025 100644 --- a/cmd/explaintest/r/new_character_set.result +++ b/cmd/explaintest/r/new_character_set.result @@ -66,27 +66,6 @@ b d select hex(b), hex(d) from t; hex(b) hex(d) E4BDA0E5A5BD 7B22E6B58BE8AF95223A2022E4BDA0E5A5BD227D -show charset; -Charset Description Default collation Maxlen -ascii US ASCII ascii_bin 1 -binary binary binary 1 -gbk Chinese Internal Code Specification gbk_chinese_ci 2 -latin1 Latin1 latin1_bin 1 -utf8 UTF-8 Unicode utf8_bin 3 -utf8mb4 UTF-8 Unicode utf8mb4_bin 4 -show collation; -Collation Charset Id Default Compiled Sortlen -ascii_bin ascii 65 Yes Yes 1 -binary binary 63 Yes Yes 1 -gbk_bin gbk 87 Yes 1 -gbk_chinese_ci gbk 28 Yes Yes 1 -latin1_bin latin1 47 Yes Yes 1 -utf8_bin utf8 83 Yes Yes 1 -utf8_general_ci utf8 33 Yes 1 -utf8_unicode_ci utf8 192 Yes 1 -utf8mb4_bin utf8mb4 46 Yes Yes 1 -utf8mb4_general_ci utf8mb4 45 Yes 1 -utf8mb4_unicode_ci utf8mb4 224 Yes 1 set names utf8mb4; drop table if exists t; create table t(a blob, b char(10)); diff --git a/cmd/explaintest/r/new_character_set_builtin.result b/cmd/explaintest/r/new_character_set_builtin.result index 070e81327814b..96f903a13ee1b 100644 --- a/cmd/explaintest/r/new_character_set_builtin.result +++ b/cmd/explaintest/r/new_character_set_builtin.result @@ -398,13 +398,17 @@ a like 0xe4b880 b like 0xd2bb 1 1 1 1 select a = 0xb6fe from t; -Error 3854: Cannot convert string 'B6FE' from binary to utf8mb4 +Error 3854: Cannot convert string '\xB6\xFE' from binary to utf8mb4 select b = 0xe4ba8c from t; -Error 3854: Cannot convert string 'E4BA8C' from binary to gbk +Error 3854: Cannot convert string '\xE4\xBA\x8C' from binary to gbk select concat(a, 0xb6fe) from t; -Error 3854: Cannot convert string 'B6FE' from binary to utf8mb4 +Error 3854: Cannot convert string '\xB6\xFE' from binary to utf8mb4 select concat(b, 0xe4ba8c) from t; -Error 3854: Cannot convert string 'E4BA8C' from binary to gbk +Error 3854: Cannot convert string '\xE4\xBA\x8C' from binary to gbk +select concat(convert('a' using gbk), 0x3fff) from t; +Error 3854: Cannot convert string '?\xFF' from binary to gbk +select concat(convert('a' using gbk), 0x3fffffffffffffff) from t; +Error 3854: Cannot convert string '?\xFF\xFF\xFF\xFF\xFF...' from binary to gbk set @@tidb_enable_vectorized_expression = false; select hex(concat(a, c)), hex(concat(b, c)) from t; hex(concat(a, c)) hex(concat(b, c)) @@ -497,13 +501,13 @@ a like 0xe4b880 b like 0xd2bb 1 1 1 1 select a = 0xb6fe from t; -Error 3854: Cannot convert string 'B6FE' from binary to utf8mb4 +Error 3854: Cannot convert string '\xB6\xFE' from binary to utf8mb4 select b = 0xe4ba8c from t; -Error 3854: Cannot convert string 'E4BA8C' from binary to gbk +Error 3854: Cannot convert string '\xE4\xBA\x8C' from binary to gbk select concat(a, 0xb6fe) from t; -Error 3854: Cannot convert string 'B6FE' from binary to utf8mb4 +Error 3854: Cannot convert string '\xB6\xFE' from binary to utf8mb4 select concat(b, 0xe4ba8c) from t; -Error 3854: Cannot convert string 'E4BA8C' from binary to gbk +Error 3854: Cannot convert string '\xE4\xBA\x8C' from binary to gbk drop table if exists t; create table t (a char(20) charset utf8mb4, b char(20) charset gbk, c binary(20)); insert into t values ('一二三', '一二三', '一二三'); diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index f364a577e1071..63bdb29e7d39b 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -498,4 +498,4 @@ a b c d create table t3(a char(10), primary key (a)); insert into t3 values ('a'); select * from t3 where a > 0x80; -Error 1105: Cannot convert string '80' from binary to utf8mb4 +Error 1105: Cannot convert string '\x80' from binary to utf8mb4 diff --git a/cmd/explaintest/r/vitess_hash.result b/cmd/explaintest/r/vitess_hash.result index 1e20ff988c8f4..a6beca77cbe4e 100644 --- a/cmd/explaintest/r/vitess_hash.result +++ b/cmd/explaintest/r/vitess_hash.result @@ -19,10 +19,11 @@ explain format = 'brief' select id from t where (vitess_hash(customer_id) >> 56) id estRows task access object operator info Projection 0.50 root test.t.id └─Sort 0.50 root test.t.id - └─IndexLookUp 0.50 root - ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t, index:t_vitess_shard(vitess_hash(`customer_id`) >> 56) range:[224,224], [225,225], keep order:false, stats:pseudo - └─Selection(Probe) 0.50 cop[tikv] ge(test.t.id, 2), le(test.t.id, 5) - └─TableRowIDScan 20.00 cop[tikv] table:t keep order:false, stats:pseudo + └─Projection 0.50 root test.t.id, rightshift(vitess_hash(test.t.customer_id), 56) + └─IndexLookUp 0.50 root + ├─IndexRangeScan(Build) 20.00 cop[tikv] table:t, index:t_vitess_shard(vitess_hash(`customer_id`) >> 56) range:[224,224], [225,225], keep order:false, stats:pseudo + └─Selection(Probe) 0.50 cop[tikv] ge(test.t.id, 2), le(test.t.id, 5) + └─TableRowIDScan 20.00 cop[tikv] table:t keep order:false, stats:pseudo explain format = 'brief' select hex(vitess_hash(1123)) from t; id estRows task access object operator info Projection 10000.00 root 31B565D41BDF8CA->Column#7 diff --git a/cmd/explaintest/run-tests.sh b/cmd/explaintest/run-tests.sh index 5fa76d57f80d9..e05fc2ea6bf35 100755 --- a/cmd/explaintest/run-tests.sh +++ b/cmd/explaintest/run-tests.sh @@ -28,6 +28,7 @@ record_case="" create=0 create_case="" stats="s" +collation_opt=2 set -eu trap 'set +e; PIDS=$(jobs -p); [ -n "$PIDS" ] && kill -9 $PIDS' EXIT @@ -40,6 +41,11 @@ function help_message() -h: Print this help message. + -d : \"y\" or \"Y\" for only enabling the new collation during test. + \"n\" or \"N\" for only disabling the new collation during test. + \"b\" or \"B\" for both tests [default]. + Enable/Disable the new collation during the explain test. + -s : Use tidb-server in for testing. eg. \"./run-tests.sh -s ./explaintest_tidb-server\" @@ -105,7 +111,7 @@ function extract_stats() unzip -qq s.zip } -while getopts "t:s:r:b:c:i:h:p" opt; do +while getopts "t:s:r:b:d:c:i:h:p" opt; do case $opt in t) tests="$OPTARG" @@ -131,6 +137,20 @@ while getopts "t:s:r:b:c:i:h:p" opt; do ;; esac ;; + d) + case $OPTARG in + y|Y) + collation_opt=1 + ;; + n|N) + collation_opt=0 + ;; + *) + help_messge 1>&2 + exit 1 + ;; + esac + ;; h) help_message exit 0 @@ -216,88 +236,84 @@ done port=${ports[0]} status=${ports[1]} -echo "start tidb-server, log file: $explain_test_log" -if [ "${TIDB_TEST_STORE_NAME}" = "tikv" ]; then - $tidb_server -P "$port" -status "$status" -config config.toml -store tikv -path "${TIKV_PATH}" > $explain_test_log 2>&1 & - SERVER_PID=$! -else - $tidb_server -P "$port" -status "$status" -config config.toml -store unistore -path "" > $explain_test_log 2>&1 & - SERVER_PID=$! -fi -echo "tidb-server(PID: $SERVER_PID) started" - -sleep 5 - -if [ $record -eq 1 ]; then - if [ "$record_case" = 'all' ]; then - echo "record all cases" - $explain_test -port "$port" -status "$status" --record --log-level=error - else - echo "record result for case: \"$record_case\"" - $explain_test -port "$port" -status "$status" --record $record_case --log-level=error +function start_tidb_server() +{ + config_file="config.toml" + if [[ $enabled_new_collation = 0 ]]; then + config_file="disable_new_collation.toml" fi -elif [ $create -eq 1 ]; then - if [ "$create_case" = 'all' ]; then - echo "create all cases" - $explain_test -port "$port" -status "$status" --create --log-level=error + echo "start tidb-server, log file: $explain_test_log" + if [ "${TIDB_TEST_STORE_NAME}" = "tikv" ]; then + $tidb_server -P "$port" -status "$status" -config $config_file -store tikv -path "${TIKV_PATH}" > $explain_test_log 2>&1 & + SERVER_PID=$! else - echo "create result for case: \"$create_case\"" - $explain_test -port "$port" -status "$status" --create $create_case --log-level=error + $tidb_server -P "$port" -status "$status" -config $config_file -store unistore -path "" > $explain_test_log 2>&1 & + SERVER_PID=$! fi -else - if [ -z "$tests" ]; then - echo "run all explain test cases" + echo "tidb-server(PID: $SERVER_PID) started" +} + +function run_explain_test() +{ + coll_disabled="false" + coll_msg="enabled new collation" + if [[ $enabled_new_collation = 0 ]]; then + coll_disabled="true" + coll_msg="disabled new collation" + fi + if [ $record -eq 1 ]; then + if [ "$record_case" = 'all' ]; then + echo "record all cases" + $explain_test -port "$port" -status "$status" --collation-disable=$coll_disabled --record --log-level=error + else + echo "record result for case: \"$record_case\"" + $explain_test -port "$port" -status "$status" --collation-disable=$coll_disabled --record $record_case --log-level=error + fi + elif [ $create -eq 1 ]; then + if [ "$create_case" = 'all' ]; then + echo "create all cases" + $explain_test -port "$port" -status "$status" --collation-disable=$coll_disabled --create --log-level=error + else + echo "create result for case: \"$create_case\"" + $explain_test -port "$port" -status "$status" --collation-disable=$coll_disabled --create $create_case --log-level=error + fi else - echo "run explain test cases: $tests" + if [ -z "$tests" ]; then + echo "run all explain test cases ($coll_msg)" + else + echo "run explain test cases($coll_msg): $tests" + fi + $explain_test -port "$port" -status "$status" --collation-disable=$coll_disabled --log-level=error $tests fi - $explain_test -port "$port" -status "$status" --log-level=error $tests -fi +} -race=`grep 'DATA RACE' $explain_test_log || true` -if [ ! -z "$race" ]; then - echo "tidb-server DATA RACE!" - cat $explain_test_log - exit 1 -fi +function check_data_race() { + race=`grep 'DATA RACE' $explain_test_log || true` + if [ ! -z "$race" ]; then + echo "tidb-server DATA RACE!" + cat $explain_test_log + exit 1 + fi +} -kill -9 $SERVER_PID +enabled_new_collation="" -echo "run collation tests when new-collation disabled" -echo "start tidb-server, log file: $explain_test_log" -if [ "${TIDB_TEST_STORE_NAME}" = "tikv" ]; then - $tidb_server -P "$port" -status "$status" -config disable_new_collation.toml -store tikv -path "${TIKV_PATH}" >> $explain_test_log 2>&1 & - SERVER_PID=$! -else - $tidb_server -P "$port" -status "$status" -config disable_new_collation.toml -store unistore -path "" >> $explain_test_log 2>&1 & - SERVER_PID=$! +if [[ $collation_opt = 0 || $collation_opt = 2 ]]; then + enabled_new_collation=0 + start_tidb_server + sleep 5 + run_explain_test + kill -9 $SERVER_PID + check_data_race fi -echo "tidb-server(PID: $SERVER_PID) started" - -sleep 5 -if [ $record -eq 1 ]; then - if [ "$record_case" = 'all' ]; then - echo "record all collation cases" - $explain_test -port "$port" -status "$status" --record --log-level=error --collation-disable=true - else - echo "record result for case: \"$record_case\"" - $explain_test -port "$port" -status "$status" --collation-disable=true --record $record_case --log-level=error - fi -elif [ $create -eq 1 ]; then - if [ "$create_case" = 'all' ]; then - echo "create all collation cases" - $explain_test -port "$port" -status "$status" --create --log-level=error --collation-disable=true - else - echo "create result for case: \"$create_case\"" - $explain_test -port "$port" -status "$status" --collation-disable=true --create $create_case --log-level=error - fi -else - if [ -z "$tests" ]; then - echo "run all collation explain test cases" - else - echo "run explain test cases: $tests" - fi - $explain_test -port "$port" -status "$status" --collation-disable=true --log-level=error $tests +if [[ $collation_opt = 1 || $collation_opt = 2 ]]; then + enabled_new_collation=1 + start_tidb_server + sleep 5 + run_explain_test + kill -9 $SERVER_PID + check_data_race fi -echo "explaintest end" +echo "explaintest passed!" diff --git a/cmd/explaintest/t/collation_misc.test b/cmd/explaintest/t/collation_misc.test new file mode 100644 index 0000000000000..443c6c4106cc0 --- /dev/null +++ b/cmd/explaintest/t/collation_misc.test @@ -0,0 +1,65 @@ +# prepare database +create database collation_misc; +use collation_misc; + +# ChangingCharsetToUtf8 +create table t1(a varchar(20) charset utf8); +insert into t1 values ("t1_value"); +alter table t1 collate uTf8mB4_uNiCoDe_Ci charset Utf8mB4 charset uTF8Mb4 collate UTF8MB4_BiN; +alter table t1 modify column a varchar(20) charset utf8mb4; +select * from t1; +create table t(a varchar(20) charset latin1); +insert into t values ("t_value"); +alter table t modify column a varchar(20) charset latin1; +select * from t; +--error 8200 +alter table t modify column a varchar(20) charset utf8; +--error 8200 +alter table t modify column a varchar(20) charset utf8mb4; +--error 8200 +alter table t modify column a varchar(20) charset utf8 collate utf8_bin; +--error 8200 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8mb4_general_ci; +--error 1273 +alter table t modify column a varchar(20) charset utf8mb4 collate utf8bin; +--error 1302, 1273 +alter table t collate LATIN1_GENERAL_CI charset utf8 collate utf8_bin; +--error 1253, 1273 +alter table t collate LATIN1_GENERAL_CI collate UTF8MB4_UNICODE_ci collate utf8_bin; + +# TestCharsetDatabase +create database if not exists cd_test_utf8 CHARACTER SET utf8 COLLATE utf8_bin; +--error 1273 +create database if not exists cd_test_latin1 CHARACTER SET latin1 COLLATE latin1_swedish_ci; +use cd_test_utf8; +select @@character_set_database; +select @@collation_database; +--error 1049 +use cd_test_latin1; +select @@character_set_database; +select @@collation_database; + +# DefaultDBAfterDropCurDB +--error 1273 +create database if not exists test_db CHARACTER SET latin1 COLLATE latin1_swedish_ci; + +# CollationUnion +with cte as (select cast('2010-09-09' as date) a union select '2010-09-09 ') select count(*) from cte; + +# Issue26989 +set names utf8mb4 collate utf8mb4_general_ci; +select position('a' in 'AA'); +select locate('a', 'AA'); +select locate('a', 'a'); +set names utf8mb4; + +# CharacterSetCollations +SELECT default_collate_name, maxlen FROM information_schema.character_sets ORDER BY character_set_name; +SELECT character_set_name, id, sortlen FROM information_schema.collations ORDER BY collation_name, id; +select * from information_schema.COLLATION_CHARACTER_SET_APPLICABILITY where COLLATION_NAME='utf8mb4_bin'; + +# charset +show charset; +show collation; + +use test; diff --git a/cmd/explaintest/t/collation_pointget.test b/cmd/explaintest/t/collation_pointget.test new file mode 100644 index 0000000000000..ea4cdca6f05c9 --- /dev/null +++ b/cmd/explaintest/t/collation_pointget.test @@ -0,0 +1,129 @@ +# prepare database +create database collation_point_get; +use collation_point_get; + +# pointGet IndexLookupChar +drop table if exists t; +create table t(a char(2), b char(2), index idx_1(a)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +select * from t where a = "aab"; +# Test query with table alias +select * from t tmp where a = "aa"; +select * from t tmp where a = "aab"; +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +drop table if exists t; +create table t(a char(2) binary, b char(2), index idx_1(a)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +select * from t where a = ""; +select * from t where a = " "; +select * from t where a = " "; +select * from t where a = " "; + +# pointGet AliasTableCharPK +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select * from t tmp where a = "aa"; +select * from t tmp where a = "aab"; +truncate table t; +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +select * from t tmp where a = "a "; +select * from t tmp where a = "a "; +# Test CHAR BINARY. +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t tmp where a = "a"; +select * from t tmp where a = "a "; +select * from t tmp where a = "a "; +select * from t tmp where a = ""; +select * from t tmp where a = " "; +select * from t tmp where a = " "; +# Test both wildcard and column name exist in select field list +drop table if exists t; +create table t(a char(2) primary key, b char(2)); +insert into t values("aa", "bb"); +select *, a from t tmp where a = "aa"; +# Test using table alias in field list +select tmp.* from t tmp where a = "aa"; +select tmp.a, tmp.b from t tmp where a = "aa"; +select tmp.*, tmp.a, tmp.b from t tmp where a = "aa"; +select tmp.* from t tmp where a = "aab"; +select tmp.a, tmp.b from t tmp where a = "aab"; +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +select tmp.*, tmp.a, tmp.b from t tmp where a = "aab"; +# Test using table alias in where clause +select * from t tmp where tmp.a = "aa"; +select a, b from t tmp where tmp.a = "aa"; +select *, a, b from t tmp where tmp.a = "aa"; +# Unknown table name in where clause and field list +--error 1054 +select a from t where xxxxx.a = "aa"; +--error 1054 +select xxxxx.a from t where a = "aa"; +# When an alias is provided, it completely hides the actual name of the table. +--error 1054 +select a from t tmp where t.a = "aa"; +--error 1054 +select t.a from t tmp where a = "aa"; +--error 1051 +select t.* from t tmp where a = "aa"; + +# PointGetCharPK +drop table if exists t; +create table t(a char(4) primary key, b char(4)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +select * from t where a = "aab"; +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +# Test CHAR BINARY. +drop table if exists t; +create table t(a char(2) binary primary key, b char(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +select * from t where a = ""; +select * from t where a = " "; +select * from t where a = " "; + +# PointGetVarcharPK +drop table if exists t; +create table t(a varchar(2) primary key, b varchar(2)); +insert into t values("aa", "bb"); +select * from t where a = "aa"; +select * from t where a = "aab"; +truncate table t; +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +# Test VARCHAR BINARY. +drop table if exists t; +create table t(a varchar(2) binary primary key, b varchar(2)); +insert into t values(" ", " "); +insert into t values("a ", "b "); +select * from t where a = "a"; +select * from t where a = "a "; +select * from t where a = "a "; +select * from t where a = ""; +select * from t where a = " "; +select * from t where a = " "; + +use mysql; diff --git a/cmd/explaintest/t/common_collation.test b/cmd/explaintest/t/common_collation.test index bf5751f2c7d53..8f19d9d1a3e27 100644 --- a/cmd/explaintest/t/common_collation.test +++ b/cmd/explaintest/t/common_collation.test @@ -8,6 +8,20 @@ select coercibility(convert(concat(a, b) using utf8mb4) collate utf8mb4_general_ select coercibility(convert('a' using utf8mb4)); select coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci); +# test for coercibility and collation with json type +# see details from https://github.com/pingcap/tidb/issues/31541 and https://github.com/pingcap/tidb/issues/31320#issuecomment-1010599311 +--disable_warnings +drop table if exists t; +--enable_warnings +create table t (a char(20), b blob(100), c text, d json, e timestamp, f set('a一','b二','c三','d四'), g text, h enum('a一','b二','c三','d四') default 'c三'); +insert into t values ('你好', '你好', '你好', '{\"测试\": \"你好\"}', '2018-10-13', 1, '你好', 'a一'); +select coercibility(a), coercibility(b), coercibility(c), coercibility(d), coercibility(e), coercibility(f), coercibility(g), coercibility(h) from t; +select collation(d), collation(upper(d)), collation(elt(1, d, 0x12)), collation(elt(1, elt(1, d, 0x12), 0x12)), collation(elt(1, d, b)) from t; +drop table t; +create table t(a binary, b json, c char charset gbk); +insert into t values ('a', '{"a":"b"}', 'a'); +select collation(concat(a, b)), collation(concat(b, a)), collation(concat(0x61, b)), collation(concat(b, 0x61)), collation(concat(c, b)), collation(concat(b, c)) from t; + # test greatest and least function with collation. DROP TABLE IF EXISTS t2; CREATE TABLE t2 ( diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index f71b05c59f5c7..4738d13cb9da1 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -266,4 +266,30 @@ create table t ( c_int int, c_str varchar(40) character set utf8 collate utf8_ge replace into t (c_int, c_str) values (9, "beautiful hermann"); select reverse(c_str) from t use index(idx); +drop table if exists t1; +drop table if exists t2; +create table t1 (c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci, c_datetime datetime, c_timestamp timestamp, c_double double, c_decimal decimal(12, 6), c_enum enum('blue','green','red','yellow','white','orange','purple'), primary key (c_datetime) , key(c_int) , key(c_datetime) , key((c_int + 1)), key((c_int -1)), key((lower(c_str))), key((md5(c_str))), key((reverse(c_str))), key((upper(c_str)))); +create table t2 like t1; +insert into t1 values(11, 'loving grothendieck', '2020-02-02 19:25:49', '2020-03-27 15:17:14', 3.269, 1.851000, 'white' ); +insert into t1 values(11, 'quirky kapitsa' , '2020-06-21 03:55:31', '2020-02-29 17:02:48', 6.94, 1.851000, 'yellow'); +insert into t1 values( 7, 'boring bouman' , '2020-05-10 00:01:04', '2020-02-01 20:18:00', 84.096168, 6.996000, 'white' ); +insert into t2 values( 11, 'wizardly antonelli', '2020-01-30 17:27:17', '2020-01-01 10:05:31', 6.886177, 6.332000, 'green' ); +insert into t2 values( 2, 'angry kapitsa' , '2020-03-30 05:09:44', '2020-02-15 00:36:52', 95.798378, 3.118000, 'blue' ); +insert into t2 values( 7, 'dreamy shamir' , '2020-05-28 14:13:42', '2020-06-02 07:23:22', 26.623227, 3.105000, 'orange'); +begin; +delete from t2 where c_decimal > c_double/2 order by c_int, c_str, c_double, c_decimal limit 1; +desc format='brief' select t2.c_enum from t2,t1 where t1.c_int - 1 = t2.c_int - 1 order by t2.c_enum; +select t2.c_enum from t2,t1 where t1.c_int - 1 = t2.c_int - 1 order by t2.c_enum; +drop table t1, t2; + drop table t; + +drop table if exists t1,t2; +create table t1 (c_int int, c_str varchar(40) ,primary key (c_int) , key(c_str(36)) , key((c_int + 1))) partition by hash (c_int) partitions 4 ; +create table t2 like t1 ; +insert into t1 values (1, 'sleepy kowalevski'); +insert into t2 values (3, 'unruffled chaplygin'); +select (select t2.c_str from t2 where t2.c_int + 1 = 4 order by t2.c_str) x from t1; +select (select t2.c_str from t2 where t2.c_int = 3 order by t2.c_str) x from t1; + +drop table t1,t2; diff --git a/cmd/explaintest/t/explain_shard_index.test b/cmd/explaintest/t/explain_shard_index.test new file mode 100644 index 0000000000000..4264ecfa47796 --- /dev/null +++ b/cmd/explaintest/t/explain_shard_index.test @@ -0,0 +1,22 @@ +use test; +drop table if exists test3, test5; +create table test3(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a)); +create table test5(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a,b)); + +explain format=brief select * from test3 where a=100; +explain format=brief select * from test3 where a=100 and (b = 100 or b = 200); +explain format=brief select * from test3 where tidb_shard(a) = 8; +explain format=brief select * from test3 where a=100 or b = 200; +explain format=brief select * from test3 where a=100 or a = 300; +explain format=brief select * from test3 where a=100 or a = 300 or a > 997; +explain format=brief select * from test3 where ((a=100 and b = 100) or a = 200) and b = 300; +explain format=brief select * from test3 where a = b; +explain format=brief select * from test3 where a = b and b = 100; +explain format=brief select * from test5 where a=100 and b = 100; +explain format=brief select * from test5 where (a=100 and b = 100) or (a=200 and b = 200); +explain format=brief select a+b from test5 where (a, b) in ((100, 100), (200, 200)); +explain format=brief SELECT * FROM test3 WHERE a IN (100); +explain format=brief SELECT * FROM test3 WHERE a IN (100, 200, 300); + +drop table if exists test3, test5; + diff --git a/cmd/explaintest/t/new_character_set.test b/cmd/explaintest/t/new_character_set.test index a608bda7622f5..7de21592b89de 100644 --- a/cmd/explaintest/t/new_character_set.test +++ b/cmd/explaintest/t/new_character_set.test @@ -46,8 +46,6 @@ create table t (b blob, d json); insert into t values ('你好', '{"测试": "你好"}'); select b, d from t; select hex(b), hex(d) from t; -show charset; -show collation; set names utf8mb4; drop table if exists t; diff --git a/cmd/explaintest/t/new_character_set_builtin.test b/cmd/explaintest/t/new_character_set_builtin.test index e1682a48bdd31..93f160832ce01 100644 --- a/cmd/explaintest/t/new_character_set_builtin.test +++ b/cmd/explaintest/t/new_character_set_builtin.test @@ -192,14 +192,18 @@ select hex(instr(a, 0xe4b880)), hex(instr(b, 0xd2bb)) from t; select hex(position(a in 0xe4b880)), hex(position(b in 0xd2bb)) from t; select a like 0xe4b880, b like 0xd2bb from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select a = 0xb6fe from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select b = 0xe4ba8c from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select concat(a, 0xb6fe) from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select concat(b, 0xe4ba8c) from t; +--error 3854 +select concat(convert('a' using gbk), 0x3fff) from t; +--error 3854 +select concat(convert('a' using gbk), 0x3fffffffffffffff) from t; set @@tidb_enable_vectorized_expression = false; select hex(concat(a, c)), hex(concat(b, c)) from t; @@ -218,13 +222,13 @@ select hex(instr(a, 0xe4b880)), hex(instr(b, 0xd2bb)) from t; select hex(position(a in 0xe4b880)), hex(position(b in 0xd2bb)) from t; select a like 0xe4b880, b like 0xd2bb from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select a = 0xb6fe from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select b = 0xe4ba8c from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select concat(a, 0xb6fe) from t; ---error ER_CANNOT_CONVERT_STRING +--error 3854 select concat(b, 0xe4ba8c) from t; -- test for builtin function aes_encrypt() diff --git a/cmd/explaintest/t/new_character_set_invalid.test b/cmd/explaintest/t/new_character_set_invalid.test index 34031d0b83ef8..eaed9ba78c518 100644 --- a/cmd/explaintest/t/new_character_set_invalid.test +++ b/cmd/explaintest/t/new_character_set_invalid.test @@ -2,11 +2,11 @@ set @@sql_mode = 'strict_trans_tables'; drop table if exists t; create table t (a varchar(255) charset gbk, b varchar(255) charset ascii, c varchar(255) charset utf8); insert into t values ('中文', 'asdf', '字符集'); --- error 1366: Incorrect string value '\xC3\x80' for column 'a' +-- error 1366 insert into t values ('À', 'ø', '😂'); --- error 1366: Incorrect string value '\xC3\x80\xE4\xB8\xAD\xE6...' for column 'a' +-- error 1366 insert into t values ('中文À中文', 'asdføfdsa', '字符集😂字符集'); --- error 1366: Incorrect string value '\xFF\xFF' for column 'a' +-- error 1366 insert into t values (0x4040ffff, 0x4040ffff, 0x4040ffff); select * from t; diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index c53aca600e2b9..62d1a49ce629b 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -56,10 +56,10 @@ SELECT * from t a, t2 b; SELECT * from t as a, t2 as b; SELECT * from t a left join t2 b on a.c1 = b.c1; ---error ER_DUP_FIELDNAME +--error 1060 SELECT * from (SELECT 1, 1) as a; ---error ER_DUP_FIELDNAME +--error 1060 SELECT * from (SELECT * FROM t, t2) as a; # Select bool field @@ -247,5 +247,5 @@ SELECT a, b, c, d FROM precise_types; create table t3(a char(10), primary key (a)); insert into t3 values ('a'); ---error ER_CANNOT_CONVERT_STRING +--error 1105 select * from t3 where a > 0x80; diff --git a/config/config.go b/config/config.go index da01925e79faf..eba9641393917 100644 --- a/config/config.go +++ b/config/config.go @@ -569,6 +569,8 @@ type PessimisticTxn struct { DeadlockHistoryCapacity uint `toml:"deadlock-history-capacity" json:"deadlock-history-capacity"` // Whether retryable deadlocks (in-statement deadlocks) are collected to the information_schema.deadlocks table. DeadlockHistoryCollectRetryable bool `toml:"deadlock-history-collect-retryable" json:"deadlock-history-collect-retryable"` + // PessimisticAutoCommit represents if true it means the auto-commit transactions will be in pessimistic mode. + PessimisticAutoCommit AtomicBool `toml:"pessimistic-auto-commit" json:"pessimistic-auto-commit"` } // DefaultPessimisticTxn returns the default configuration for PessimisticTxn @@ -577,6 +579,7 @@ func DefaultPessimisticTxn() PessimisticTxn { MaxRetryCount: 256, DeadlockHistoryCapacity: 10, DeadlockHistoryCollectRetryable: false, + PessimisticAutoCommit: *NewAtomicBool(false), } } @@ -744,10 +747,11 @@ var defaultConf = Config{ AutoTLS: false, RSAKeySize: 4096, }, - DeprecateIntegerDisplayWidth: false, - EnableEnumLengthLimit: true, - StoresRefreshInterval: defTiKVCfg.StoresRefreshInterval, - EnableForwarding: defTiKVCfg.EnableForwarding, + DeprecateIntegerDisplayWidth: false, + EnableEnumLengthLimit: true, + StoresRefreshInterval: defTiKVCfg.StoresRefreshInterval, + EnableForwarding: defTiKVCfg.EnableForwarding, + NewCollationsEnabledOnFirstBootstrap: true, } var ( diff --git a/config/config.toml.example b/config/config.toml.example index 6a54dd3230462..4128254492606 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -109,7 +109,7 @@ repair-table-list = [] max-server-connections = 0 # Whether new collations are enabled, as indicated by its name, this configuration entry take effect ONLY when a TiDB cluster bootstraps for the first time. -new_collations_enabled_on_first_bootstrap = false +new_collations_enabled_on_first_bootstrap = true # Don't register information of this TiDB to etcd, so this instance of TiDB won't appear in the services like dashboard. # This option is useful when you want to embed TiDB into your service(i.e. use TiDB as a library). @@ -460,6 +460,9 @@ deadlock-history-capacity = 10 # Whether retryable deadlocks (in-statement deadlocks) are collected to the information_schema.deadlocks table. deadlock-history-collect-retryable = false +# If true it means the auto-commit transactions will be in pessimistic mode. +pessimistic-auto-commit = false + # experimental section controls the features that are still experimental: their semantics, # interfaces are subject to change, using these features in the production environment is not recommended. [experimental] diff --git a/config/config_test.go b/config/config_test.go index 3f8d130b2bd3a..913dbe088f0be 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -252,6 +252,7 @@ spilled-file-encryption-method = "plaintext" [pessimistic-txn] deadlock-history-capacity = 123 deadlock-history-collect-retryable = true +pessimistic-auto-commit = true [top-sql] receiver-address = "127.0.0.1:10100" [status] @@ -308,6 +309,7 @@ grpc-max-send-msg-size = 40960 require.Equal(t, uint64(30), conf.StoresRefreshInterval) require.Equal(t, uint(123), conf.PessimisticTxn.DeadlockHistoryCapacity) require.True(t, conf.PessimisticTxn.DeadlockHistoryCollectRetryable) + require.True(t, conf.PessimisticTxn.PessimisticAutoCommit.Load()) require.Equal(t, "127.0.0.1:10100", conf.TopSQL.ReceiverAddress) require.True(t, conf.Experimental.AllowsExpressionIndex) require.Equal(t, uint(20), conf.Status.GRPCKeepAliveTime) @@ -643,7 +645,7 @@ func TestSecurityValid(t *testing.T) { func TestTcpNoDelay(t *testing.T) { c1 := NewConfig() - //check default value + // check default value require.True(t, c1.Performance.TCPNoDelay) } diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index c3295b1d518d0..9f6a7ac4425e6 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -18,69 +18,41 @@ import ( "context" "fmt" "math" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain/infosync" - "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/gcworker" - "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/gcutil" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testAttributesDDLSerialSuite{}) - -type testAttributesDDLSerialSuite struct{} - -func (s *testAttributesDDLSerialSuite) TestAlterTableAttributes(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestAlterTableAttributes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table alter_t (c int);`) // normal cases - _, err = tk.Exec(`alter table alter_t attributes="merge_option=allow";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_t attributes="merge_option=allow,key=value";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_t attributes="merge_option=allow";`) + tk.MustExec(`alter table alter_t attributes="merge_option=allow,key=value";`) // space cases - _, err = tk.Exec(`alter table alter_t attributes=" merge_option=allow ";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_t attributes=" merge_option = allow , key = value ";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_t attributes=" merge_option=allow ";`) + tk.MustExec(`alter table alter_t attributes=" merge_option = allow , key = value ";`) // without equal - _, err = tk.Exec(`alter table alter_t attributes " merge_option=allow ";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_t attributes " merge_option=allow , key=value ";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_t attributes " merge_option=allow ";`) + tk.MustExec(`alter table alter_t attributes " merge_option=allow , key=value ";`) + } -func (s *testAttributesDDLSerialSuite) TestAlterTablePartitionAttributes(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestAlterTablePartitionAttributes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table alter_p (c int) PARTITION BY RANGE (c) ( @@ -91,37 +63,22 @@ PARTITION BY RANGE (c) ( );`) // normal cases - _, err = tk.Exec(`alter table alter_p partition p0 attributes="merge_option=allow";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_p partition p1 attributes="merge_option=allow,key=value";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_p partition p0 attributes="merge_option=allow";`) + tk.MustExec(`alter table alter_p partition p1 attributes="merge_option=allow,key=value";`) // space cases - _, err = tk.Exec(`alter table alter_p partition p2 attributes=" merge_option=allow ";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_p partition p3 attributes=" merge_option = allow , key = value ";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_p partition p2 attributes=" merge_option=allow ";`) + tk.MustExec(`alter table alter_p partition p3 attributes=" merge_option = allow , key = value ";`) // without equal - _, err = tk.Exec(`alter table alter_p partition p1 attributes " merge_option=allow ";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table alter_p partition p1 attributes " merge_option=allow , key=value ";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table alter_p partition p1 attributes " merge_option=allow ";`) + tk.MustExec(`alter table alter_p partition p1 attributes " merge_option=allow , key=value ";`) } -func (s *testAttributesDDLSerialSuite) TestTruncateTable(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestTruncateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table truncate_t (c int) PARTITION BY RANGE (c) ( @@ -130,58 +87,44 @@ PARTITION BY RANGE (c) ( );`) // add attributes - _, err = tk.Exec(`alter table truncate_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table truncate_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table truncate_t attributes="key=value";`) + tk.MustExec(`alter table truncate_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // truncate table - _, err = tk.Exec(`truncate table truncate_t;`) - c.Assert(err, IsNil) + tk.MustExec(`truncate table truncate_t;`) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) + require.Len(t, rows1, 2) // check table truncate_t's attribute - c.Assert(rows1[0][0], Equals, "schema/test/truncate_t") - c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Not(Equals), rows[0][3]) + require.Equal(t, "schema/test/truncate_t", rows1[0][0]) + require.Equal(t, `"key=value"`, rows1[0][2]) + require.NotEqual(t, rows[0][3], rows1[0][3]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/truncate_t/p0") - c.Assert(rows1[1][2], Equals, `"key1=value1"`) - c.Assert(rows1[1][3], Not(Equals), rows[1][3]) + require.Equal(t, "schema/test/truncate_t/p0", rows1[1][0]) + require.Equal(t, `"key1=value1"`, rows1[1][2]) + require.NotEqual(t, rows[1][3], rows1[1][3]) // test only table tk.MustExec(`create table truncate_ot (c int);`) // add attribute - _, err = tk.Exec(`alter table truncate_ot attributes="key=value";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table truncate_ot attributes="key=value";`) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows2), Equals, 3) + require.Len(t, rows2, 3) // truncate table - _, err = tk.Exec(`truncate table truncate_ot;`) - c.Assert(err, IsNil) + tk.MustExec(`truncate table truncate_ot;`) rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows3), Equals, 3) + require.Len(t, rows3, 3) // check table truncate_ot's attribute - c.Assert(rows3[0][0], Equals, "schema/test/truncate_ot") - c.Assert(rows3[0][2], Equals, `"key=value"`) - c.Assert(rows3[0][3], Not(Equals), rows2[0][3]) + require.Equal(t, "schema/test/truncate_ot", rows3[0][0]) + require.Equal(t, `"key=value"`, rows3[0][2]) + require.NotEqual(t, rows2[0][3], rows3[0][3]) } -func (s *testAttributesDDLSerialSuite) TestRenameTable(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestRenameTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table rename_t (c int) PARTITION BY RANGE (c) ( @@ -190,58 +133,45 @@ PARTITION BY RANGE (c) ( );`) // add attributes - _, err = tk.Exec(`alter table rename_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table rename_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table rename_t attributes="key=value";`) + tk.MustExec(`alter table rename_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // rename table - _, err = tk.Exec(`rename table rename_t to rename_t1;`) - c.Assert(err, IsNil) + tk.MustExec(`rename table rename_t to rename_t1;`) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) + require.Len(t, rows1, 2) // check table rename_t1's attribute - c.Assert(rows1[0][0], Equals, "schema/test/rename_t1") - c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Equals, rows[0][3]) + require.Equal(t, "schema/test/rename_t1", rows1[0][0]) + require.Equal(t, `"key=value"`, rows1[0][2]) + require.Equal(t, rows[0][3], rows1[0][3]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/rename_t1/p0") - c.Assert(rows1[1][2], Equals, `"key1=value1"`) - c.Assert(rows1[1][3], Equals, rows[1][3]) + require.Equal(t, "schema/test/rename_t1/p0", rows1[1][0]) + require.Equal(t, `"key1=value1"`, rows1[1][2]) + require.Equal(t, rows[1][3], rows1[1][3]) // test only table tk.MustExec(`create table rename_ot (c int);`) // add attribute - _, err = tk.Exec(`alter table rename_ot attributes="key=value";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table rename_ot attributes="key=value";`) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows2), Equals, 3) + require.Len(t, rows2, 3) // rename table - _, err = tk.Exec(`rename table rename_ot to rename_ot1;`) - c.Assert(err, IsNil) + tk.MustExec(`rename table rename_ot to rename_ot1;`) + rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows3), Equals, 3) + require.Len(t, rows3, 3) // check table rename_ot1's attribute - c.Assert(rows3[0][0], Equals, "schema/test/rename_ot1") - c.Assert(rows3[0][2], Equals, `"key=value"`) - c.Assert(rows3[0][3], Equals, rows2[0][3]) + require.Equal(t, "schema/test/rename_ot1", rows3[0][0]) + require.Equal(t, `"key=value"`, rows3[0][2]) + require.Equal(t, rows2[0][3], rows3[0][3]) } -func (s *testAttributesDDLSerialSuite) TestRecoverTable(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestRecoverTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table recover_t (c int) PARTITION BY RANGE (c) ( @@ -255,47 +185,36 @@ PARTITION BY RANGE (c) ( // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. - err = gcutil.EnableGC(tk.Se) - c.Assert(err, IsNil) + require.NoError(t, gcutil.EnableGC(tk.Session())) // add attributes - _, err = tk.Exec(`alter table recover_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table recover_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table recover_t attributes="key=value";`) + tk.MustExec(`alter table recover_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // drop table - _, err = tk.Exec(`drop table recover_t;`) - c.Assert(err, IsNil) + tk.MustExec(`drop table recover_t;`) // recover table - _, err = tk.Exec(`recover table recover_t;`) - c.Assert(err, IsNil) + tk.MustExec(`recover table recover_t;`) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) + require.Len(t, rows1, 2) // check table recover_t's attribute - c.Assert(rows1[0][0], Equals, "schema/test/recover_t") - c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Equals, rows[0][3]) + require.Equal(t, "schema/test/recover_t", rows1[0][0]) + require.Equal(t, `"key=value"`, rows1[0][2]) + require.Equal(t, rows[0][3], rows1[0][3]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/recover_t/p0") - c.Assert(rows1[1][2], Equals, `"key1=value1"`) - c.Assert(rows1[1][3], Equals, rows[1][3]) + require.Equal(t, "schema/test/recover_t/p0", rows1[1][0]) + require.Equal(t, `"key1=value1"`, rows1[1][2]) + require.Equal(t, rows[1][3], rows1[1][3]) } -func (s *testAttributesDDLSerialSuite) TestFlashbackTable(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestFlashbackTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table flash_t (c int) PARTITION BY RANGE (c) ( @@ -309,64 +228,52 @@ PARTITION BY RANGE (c) ( // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. - err = gcutil.EnableGC(tk.Se) - c.Assert(err, IsNil) + err = gcutil.EnableGC(tk.Session()) + require.NoError(t, err) // add attributes - _, err = tk.Exec(`alter table flash_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table flash_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table flash_t attributes="key=value";`) + tk.MustExec(`alter table flash_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // drop table - _, err = tk.Exec(`drop table flash_t;`) - c.Assert(err, IsNil) + tk.MustExec(`drop table flash_t;`) // flashback table - _, err = tk.Exec(`flashback table flash_t to flash_t1;`) - c.Assert(err, IsNil) + tk.MustExec(`flashback table flash_t to flash_t1;`) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) + require.Len(t, rows1, 2) // check table flash_t1's attribute - c.Assert(rows1[0][0], Equals, "schema/test/flash_t1") - c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Equals, rows[0][3]) + require.Equal(t, "schema/test/flash_t1", rows1[0][0]) + require.Equal(t, `"key=value"`, rows1[0][2]) + require.Equal(t, rows[0][3], rows1[0][3]) // check partition p0's attribute - c.Assert(rows1[1][0], Equals, "schema/test/flash_t1/p0") - c.Assert(rows1[1][2], Equals, `"key1=value1"`) - c.Assert(rows1[1][3], Equals, rows[1][3]) + require.Equal(t, "schema/test/flash_t1/p0", rows1[1][0]) + require.Equal(t, `"key1=value1"`, rows1[1][2]) + require.Equal(t, rows[1][3], rows1[1][3]) // truncate table - _, err = tk.Exec(`truncate table flash_t1;`) - c.Assert(err, IsNil) + tk.MustExec(`truncate table flash_t1;`) // flashback table - _, err = tk.Exec(`flashback table flash_t1 to flash_t2;`) - c.Assert(err, IsNil) + tk.MustExec(`flashback table flash_t1 to flash_t2;`) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) + require.Len(t, rows1, 2) // check table flash_t2's attribute - c.Assert(rows2[0][0], Equals, "schema/test/flash_t2") - c.Assert(rows2[0][2], Equals, `"key=value"`) - c.Assert(rows2[0][3], Equals, rows[0][3]) + require.Equal(t, "schema/test/flash_t2", rows2[0][0]) + require.Equal(t, `"key=value"`, rows2[0][2]) + require.Equal(t, rows[0][3], rows2[0][3]) // check partition p0's attribute - c.Assert(rows2[1][0], Equals, "schema/test/flash_t2/p0") - c.Assert(rows2[1][2], Equals, `"key1=value1"`) - c.Assert(rows2[1][3], Equals, rows[1][3]) + require.Equal(t, "schema/test/flash_t2/p0", rows2[1][0]) + require.Equal(t, `"key1=value1"`, rows2[1][2]) + require.Equal(t, rows[1][3], rows2[1][3]) } -func (s *testAttributesDDLSerialSuite) TestDropTable(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestDropTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table drop_t (c int) PARTITION BY RANGE (c) ( @@ -384,42 +291,33 @@ PARTITION BY RANGE (c) ( // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. - err = gcutil.EnableGC(tk.Se) - c.Assert(err, IsNil) + err = gcutil.EnableGC(tk.Session()) + require.NoError(t, err) gcWorker, err := gcworker.NewMockGCWorker(store) - c.Assert(err, IsNil) + require.NoError(t, err) // add attributes - _, err = tk.Exec(`alter table drop_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table drop_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table drop_t attributes="key=value";`) + tk.MustExec(`alter table drop_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // drop table - _, err = tk.Exec(`drop table drop_t;`) - c.Assert(err, IsNil) + tk.MustExec(`drop table drop_t;`) err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) - c.Assert(err, IsNil) + require.NoError(t, err) rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testAttributesDDLSerialSuite) TestCreateWithSameName(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestCreateWithSameName(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table recreate_t (c int) PARTITION BY RANGE (c) ( @@ -437,25 +335,22 @@ PARTITION BY RANGE (c) ( // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. - err = gcutil.EnableGC(tk.Se) - c.Assert(err, IsNil) + err = gcutil.EnableGC(tk.Session()) + require.NoError(t, err) gcWorker, err := gcworker.NewMockGCWorker(store) - c.Assert(err, IsNil) + require.NoError(t, err) // add attributes - _, err = tk.Exec(`alter table recreate_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table recreate_t partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table recreate_t attributes="key=value";`) + tk.MustExec(`alter table recreate_t partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // drop table - _, err = tk.Exec(`drop table recreate_t;`) - c.Assert(err, IsNil) + tk.MustExec(`drop table recreate_t;`) rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) tk.MustExec(`create table recreate_t (c int) PARTITION BY RANGE (c) ( @@ -463,40 +358,31 @@ PARTITION BY RANGE (c) ( PARTITION p1 VALUES LESS THAN (11) );`) // add attributes - _, err = tk.Exec(`alter table recreate_t attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table recreate_t partition p1 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table recreate_t attributes="key=value";`) + tk.MustExec(`alter table recreate_t partition p1 attributes="key1=value1";`) rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 3) + require.Len(t, rows, 3) err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) - c.Assert(err, IsNil) + require.NoError(t, err) rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // drop table - _, err = tk.Exec(`drop table recreate_t;`) - c.Assert(err, IsNil) + tk.MustExec(`drop table recreate_t;`) err = gcWorker.DeleteRanges(context.Background(), uint64(math.MaxInt64)) - c.Assert(err, IsNil) + require.NoError(t, err) rows = tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testAttributesDDLSerialSuite) TestPartition(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestPartition(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table part (c int) PARTITION BY RANGE (c) ( @@ -507,69 +393,56 @@ PARTITION BY RANGE (c) ( tk.MustExec(`create table part1 (c int);`) // add attributes - _, err = tk.Exec(`alter table part attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table part partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table part partition p1 attributes="key2=value2";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table part attributes="key=value";`) + tk.MustExec(`alter table part partition p0 attributes="key1=value1";`) + tk.MustExec(`alter table part partition p1 attributes="key2=value2";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows), Equals, 3) + require.Len(t, rows, 3) // drop partition // partition p0's attribute will be deleted - _, err = tk.Exec(`alter table part drop partition p0;`) - c.Assert(err, IsNil) + tk.MustExec(`alter table part drop partition p0;`) rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows1), Equals, 2) - c.Assert(rows1[0][0], Equals, "schema/test/part") - c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Equals, rows[0][3]) - c.Assert(rows1[1][0], Equals, "schema/test/part/p1") - c.Assert(rows1[1][2], Equals, `"key2=value2"`) - c.Assert(rows1[1][3], Equals, rows[2][3]) + require.Len(t, rows1, 2) + require.Equal(t, "schema/test/part", rows1[0][0]) + require.Equal(t, `"key=value"`, rows1[0][2]) + require.Equal(t, rows[0][3], rows1[0][3]) + require.Equal(t, "schema/test/part/p1", rows1[1][0]) + require.Equal(t, `"key2=value2"`, rows1[1][2]) + require.Equal(t, rows[2][3], rows1[1][3]) // truncate partition // partition p1's key range will be updated - _, err = tk.Exec(`alter table part truncate partition p1;`) - c.Assert(err, IsNil) + tk.MustExec(`alter table part truncate partition p1;`) rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows2), Equals, 2) - c.Assert(rows2[0][0], Equals, "schema/test/part") - c.Assert(rows2[0][2], Equals, `"key=value"`) - c.Assert(rows2[0][3], Not(Equals), rows1[0][3]) - c.Assert(rows2[1][0], Equals, "schema/test/part/p1") - c.Assert(rows2[1][2], Equals, `"key2=value2"`) - c.Assert(rows2[1][3], Not(Equals), rows1[1][3]) + require.Len(t, rows2, 2) + require.Equal(t, "schema/test/part", rows2[0][0]) + require.Equal(t, `"key=value"`, rows2[0][2]) + require.NotEqual(t, rows1[0][3], rows2[0][3]) + require.Equal(t, "schema/test/part/p1", rows2[1][0]) + require.Equal(t, `"key2=value2"`, rows2[1][2]) + require.NotEqual(t, rows1[1][3], rows2[1][3]) // exchange partition // partition p1's attribute will be exchanged to table part1 - _, err = tk.Exec(`set @@tidb_enable_exchange_partition=1;`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table part exchange partition p1 with table part1;`) - c.Assert(err, IsNil) + tk.MustExec(`set @@tidb_enable_exchange_partition=1;`) + tk.MustExec(`alter table part exchange partition p1 with table part1;`) rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() - c.Assert(len(rows3), Equals, 2) - c.Assert(rows3[0][0], Equals, "schema/test/part") - c.Assert(rows3[0][2], Equals, `"key=value"`) - c.Assert(rows3[0][3], Equals, rows2[0][3]) - c.Assert(rows3[1][0], Equals, "schema/test/part1") - c.Assert(rows3[1][2], Equals, `"key2=value2"`) - c.Assert(rows3[1][3], Equals, rows2[1][3]) + require.Len(t, rows3, 2) + require.Equal(t, "schema/test/part", rows3[0][0]) + require.Equal(t, `"key=value"`, rows3[0][2]) + require.Equal(t, rows2[0][3], rows3[0][3]) + require.Equal(t, "schema/test/part1", rows3[1][0]) + require.Equal(t, `"key2=value2"`, rows3[1][2]) + require.Equal(t, rows2[1][3], rows3[1][3]) } -func (s *testAttributesDDLSerialSuite) TestDropSchema(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestDropSchema(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table drop_s1 (c int) PARTITION BY RANGE (c) ( @@ -579,34 +452,24 @@ PARTITION BY RANGE (c) ( tk.MustExec(`create table drop_s2 (c int);`) // add attributes - _, err = tk.Exec(`alter table drop_s1 attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table drop_s1 partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table drop_s2 attributes="key=value";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table drop_s1 attributes="key=value";`) + tk.MustExec(`alter table drop_s1 partition p0 attributes="key1=value1";`) + tk.MustExec(`alter table drop_s2 attributes="key=value";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows() - c.Assert(len(rows), Equals, 3) + require.Len(t, rows, 3) // drop database - _, err = tk.Exec(`drop database test`) - c.Assert(err, IsNil) + tk.MustExec(`drop database test`) rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testAttributesDDLSerialSuite) TestDefaultKeyword(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - _, err = infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err := store.Close() - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, store) +func TestDefaultKeyword(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + _, err := infosync.GlobalInfoSyncerInit(context.Background(), dom.DDL().GetID(), dom.ServerID, dom.GetEtcdClient(), true) + require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table def (c int) PARTITION BY RANGE (c) ( @@ -615,20 +478,16 @@ PARTITION BY RANGE (c) ( );`) // add attributes - _, err = tk.Exec(`alter table def attributes="key=value";`) - c.Assert(err, IsNil) - _, err = tk.Exec(`alter table def partition p0 attributes="key1=value1";`) - c.Assert(err, IsNil) + tk.MustExec(`alter table def attributes="key=value";`) + tk.MustExec(`alter table def partition p0 attributes="key1=value1";`) rows := tk.MustQuery(`select * from information_schema.attributes;`).Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) // reset the partition p0's attribute - _, err = tk.Exec(`alter table def partition p0 attributes=default;`) - c.Assert(err, IsNil) + tk.MustExec(`alter table def partition p0 attributes=default;`) rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) // reset the table def's attribute - _, err = tk.Exec(`alter table def attributes=default;`) - c.Assert(err, IsNil) + tk.MustExec(`alter table def attributes=default;`) rows = tk.MustQuery(`select * from information_schema.attributes;`).Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 94f5a2ad650a3..4a17d11561951 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" decoder "github.com/pingcap/tidb/util/rowDecoder" + "github.com/pingcap/tidb/util/timeutil" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" ) @@ -536,6 +537,19 @@ func makeupDecodeColMap(sessCtx sessionctx.Context, t table.Table) (map[int64]de return decodeColMap, nil } +func setSessCtxLocation(sctx sessionctx.Context, info *reorgInfo) error { + // It is set to SystemLocation to be compatible with nil LocationInfo. + *sctx.GetSessionVars().TimeZone = *timeutil.SystemLocation() + if info.ReorgMeta.Location != nil { + loc, err := info.ReorgMeta.Location.GetLocation() + if err != nil { + return errors.Trace(err) + } + *sctx.GetSessionVars().TimeZone = *loc + } + return nil +} + // writePhysicalTableRecord handles the "add index" or "modify/change column" reorganization state for a non-partitioned table or a partition. // For a partitioned table, it should be handled partition by partition. // @@ -606,8 +620,10 @@ func (w *worker) writePhysicalTableRecord(t table.PhysicalTable, bfWorkerType ba // Simulate the sql mode environment in the worker sessionCtx. sqlMode := reorgInfo.ReorgMeta.SQLMode sessCtx.GetSessionVars().SQLMode = sqlMode - // TODO: skip set the timezone, it will cause data inconsistency when add index, since some reorg place using the timeUtil.SystemLocation() to do the time conversion. (need a more systemic plan) - // sessCtx.GetSessionVars().TimeZone = reorgInfo.ReorgMeta.Location + if err := setSessCtxLocation(sessCtx, reorgInfo); err != nil { + return errors.Trace(err) + } + sessCtx.GetSessionVars().StmtCtx.BadNullAsWarning = !sqlMode.HasStrictMode() sessCtx.GetSessionVars().StmtCtx.TruncateAsWarning = !sqlMode.HasStrictMode() sessCtx.GetSessionVars().StmtCtx.OverflowAsWarning = !sqlMode.HasStrictMode() diff --git a/ddl/column.go b/ddl/column.go index 8352c3940553a..f4d4836227ae5 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -47,7 +47,6 @@ import ( "github.com/pingcap/tidb/util/logutil" decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/sqlexec" - "github.com/pingcap/tidb/util/timeutil" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -450,16 +449,24 @@ func onDropColumns(t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateWriteOnly: // write only -> delete only setColumnsState(colInfos, model.StateDeleteOnly) - setIndicesState(idxInfos, model.StateDeleteOnly) + if len(idxInfos) > 0 { + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + for _, idx := range tblInfo.Indices { + if !indexInfoContains(idx.ID, idxInfos) { + newIndices = append(newIndices, idx) + } + } + tblInfo.Indices = newIndices + } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfos[0].State) if err != nil { return ver, errors.Trace(err) } + job.Args = append(job.Args, indexInfosToIDList(idxInfos)) job.SchemaState = model.StateDeleteOnly case model.StateDeleteOnly: // delete only -> reorganization setColumnsState(colInfos, model.StateDeleteReorganization) - setIndicesState(idxInfos, model.StateDeleteReorganization) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfos[0].State) if err != nil { return ver, errors.Trace(err) @@ -468,17 +475,6 @@ func onDropColumns(t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateDeleteReorganization: // reorganization -> absent // All reorganization jobs are done, drop this column. - if len(idxInfos) > 0 { - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if !indexInfoContains(idx.ID, idxInfos) { - newIndices = append(newIndices, idx) - } - } - tblInfo.Indices = newIndices - } - - indexIDs := indexInfosToIDList(idxInfos) tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-delCount] setColumnsState(colInfos, model.StateNone) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfos[0].State) @@ -491,7 +487,7 @@ func onDropColumns(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) } else { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) + job.Args = append(job.Args, getPartitionIDs(tblInfo)) } default: err = errInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State) @@ -508,7 +504,9 @@ func checkDropColumns(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. var colNames []model.CIStr var ifExists []bool - err = job.DecodeArgs(&colNames, &ifExists) + // indexIds is used to make sure we don't truncate args when decoding the rawArgs. + var indexIds []int64 + err = job.DecodeArgs(&colNames, &ifExists, &indexIds) if err != nil { job.State = model.JobStateCancelled return nil, nil, 0, nil, errors.Trace(err) @@ -540,6 +538,9 @@ func checkDropColumns(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. indexInfos = append(indexInfos, idxInfos...) } job.Args = []interface{}{newColNames, newIfExists} + if len(indexIds) > 0 { + job.Args = append(job.Args, indexIds) + } return tblInfo, colInfos, len(colInfos), indexInfos, nil } @@ -589,16 +590,24 @@ func onDropColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateWriteOnly: // write only -> delete only colInfo.State = model.StateDeleteOnly - setIndicesState(idxInfos, model.StateDeleteOnly) + if len(idxInfos) > 0 { + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + for _, idx := range tblInfo.Indices { + if !indexInfoContains(idx.ID, idxInfos) { + newIndices = append(newIndices, idx) + } + } + tblInfo.Indices = newIndices + } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfo.State) if err != nil { return ver, errors.Trace(err) } + job.Args = append(job.Args, indexInfosToIDList(idxInfos)) job.SchemaState = model.StateDeleteOnly case model.StateDeleteOnly: // delete only -> reorganization colInfo.State = model.StateDeleteReorganization - setIndicesState(idxInfos, model.StateDeleteReorganization) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfo.State) if err != nil { return ver, errors.Trace(err) @@ -607,17 +616,6 @@ func onDropColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateDeleteReorganization: // reorganization -> absent // All reorganization jobs are done, drop this column. - if len(idxInfos) > 0 { - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if !indexInfoContains(idx.ID, idxInfos) { - newIndices = append(newIndices, idx) - } - } - tblInfo.Indices = newIndices - } - - indexIDs := indexInfosToIDList(idxInfos) tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-1] colInfo.State = model.StateNone ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != colInfo.State) @@ -631,7 +629,7 @@ func onDropColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) { } else { // We should set related index IDs for job job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) + job.Args = append(job.Args, getPartitionIDs(tblInfo)) } default: err = errInvalidDDLJob.GenWithStackByArgs("table", tblInfo.State) @@ -647,7 +645,9 @@ func checkDropColumn(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Col } var colName model.CIStr - err = job.DecodeArgs(&colName) + // indexIds is used to make sure we don't truncate args when decoding the rawArgs. + var indexIds []int64 + err = job.DecodeArgs(&colName, &indexIds) if err != nil { job.State = model.JobStateCancelled return nil, nil, nil, errors.Trace(err) @@ -992,12 +992,7 @@ func (w *worker) doModifyColumnTypeWithData( } defer w.sessPool.put(ctx) - stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(context.Background(), true, valStr) - if err != nil { - job.State = model.JobStateCancelled - failpoint.Return(ver, err) - } - _, _, err = ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(context.Background(), stmt) + _, _, err = ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(context.Background(), nil, valStr) if err != nil { job.State = model.JobStateCancelled failpoint.Return(ver, err) @@ -1327,7 +1322,8 @@ func (w *updateColumnWorker) fetchRowColVals(txn kv.Transaction, taskRange reorg } func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, rawRow []byte) error { - _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, time.UTC, w.rowMap) + sysTZ := w.sessCtx.GetSessionVars().StmtCtx.TimeZone + _, err := w.rowDecoder.DecodeTheExistedColumnMap(w.sessCtx, handle, rawRow, sysTZ, w.rowMap) if err != nil { return errors.Trace(errCantDecodeRecord.GenWithStackByArgs("column", err)) } @@ -1350,7 +1346,7 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra val := w.rowMap[w.oldColInfo.ID] col := w.newColInfo if val.Kind() == types.KindNull && col.FieldType.Tp == mysql.TypeTimestamp && mysql.HasNotNullFlag(col.Flag) { - if v, err := expression.GetTimeCurrentTimestamp(w.sessCtx, col.Tp, int8(col.Decimal)); err == nil { + if v, err := expression.GetTimeCurrentTimestamp(w.sessCtx, col.Tp, col.Decimal); err == nil { // convert null value to timestamp should be substituted with current timestamp if NOT_NULL flag is set. w.rowMap[w.oldColInfo.ID] = v } @@ -1373,7 +1369,7 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra }) w.rowMap[w.newColInfo.ID] = newColVal - _, err = w.rowDecoder.EvalRemainedExprColumnMap(w.sessCtx, timeutil.SystemLocation(), w.rowMap) + _, err = w.rowDecoder.EvalRemainedExprColumnMap(w.sessCtx, w.rowMap) if err != nil { return errors.Trace(err) } @@ -1703,11 +1699,7 @@ func checkForNullValue(ctx context.Context, sctx sessionctx.Context, isDataTrunc } } buf.WriteString(" limit 1") - stmt, err := sctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(ctx, true, buf.String(), paramsList...) - if err != nil { - return errors.Trace(err) - } - rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(ctx, stmt) + rows, _, err := sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, nil, buf.String(), paramsList...) if err != nil { return errors.Trace(err) } @@ -1881,9 +1873,9 @@ func generateOriginDefaultValue(col *model.ColumnInfo) (interface{}, error) { if odValue == strings.ToUpper(ast.CurrentTimestamp) { if col.Tp == mysql.TypeTimestamp { - odValue = types.NewTime(types.FromGoTime(time.Now().UTC()), col.Tp, int8(col.Decimal)).String() + odValue = types.NewTime(types.FromGoTime(time.Now().UTC()), col.Tp, col.Decimal).String() } else if col.Tp == mysql.TypeDatetime { - odValue = types.NewTime(types.FromGoTime(time.Now()), col.Tp, int8(col.Decimal)).String() + odValue = types.NewTime(types.FromGoTime(time.Now()), col.Tp, col.Decimal).String() } } return odValue, nil diff --git a/ddl/column_test.go b/ddl/column_test.go index fe0e58575f1be..210d967991686 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" ) @@ -1171,9 +1170,6 @@ func (s *testColumnSuiteToVerify) TestDropColumns() { } func TestModifyColumn(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, err := mockstore.NewMockStore() require.NoError(t, err) d, err := testNewDDLAndStart( diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 2e5f88cfa038a..c8276f2d54561 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/testkit" ) @@ -376,7 +375,7 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C modifiedColumn = getModifyColumn(c, tk.Se, "test", "t", "e", false) c.Assert(modifiedColumn, NotNil) c.Assert(modifiedColumn.Tp, Equals, parser_mysql.TypeTimestamp) - tk.MustQuery("select e from t").Check(testkit.Rows("2001-11-11 00:00:00")) // the given number will be left-forward used. + tk.MustQuery("select e from t").Check(testkit.Rows("2001-11-10 16:00:00")) // the given number will be left-forward used. // integer to datetime tk.MustExec("alter table t modify f datetime") @@ -440,8 +439,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenVarcharAndNonVarchar(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("drop database if exists col_type_change_char;") tk.MustExec("create database col_type_change_char;") tk.MustExec("use col_type_change_char;") @@ -2374,3 +2371,90 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenFloatAndDouble(c tk.MustExec("alter table t modify a float(6,1)") tk.MustQuery("select a from t;").Check(testkit.Rows("36.4", "24.1")) } + +func (s *testColumnTypeChangeSuite) TestColumnTypeChangeTimestampToInt(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + // 1. modify a timestamp column to bigint + // 2. modify the bigint column to timestamp + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key auto_increment, c1 timestamp default '2020-07-10 01:05:08');") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("alter table t modify column c1 bigint;") + tk.MustQuery("select * from t").Check(testkit.Rows("1 20200710010508")) + tk.MustExec("alter table t modify c1 timestamp") + tk.MustExec("set @@session.time_zone=UTC") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-09 17:05:08")) + + // 1. modify a timestamp column to bigint + // 2. add the index + // 3. modify the bigint column to timestamp + // The current session.time_zone is '+00:00'. + tk.MustExec(`set time_zone = '+00:00'`) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key auto_increment, c1 timestamp default '2020-07-10 01:05:08', index idx(c1));") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("alter table t modify column c1 bigint;") + tk.MustExec("alter table t add index idx1(id, c1);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 20200710010508")) + tk.MustExec("admin check table t") + // change timezone + tk.MustExec("set @@session.time_zone='+5:00'") + tk.MustExec("alter table t modify c1 timestamp") + // change timezone + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("set @@session.time_zone='-8:00'") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-09 12:05:08")) + tk.MustExec("admin check table t") + // test the timezone of "default" and "system" + // The current session.time_zone is '-8:00'. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key auto_increment, c1 timestamp default '2020-07-10 01:05:08', index idx(c1));") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("alter table t modify column c1 bigint;") + tk.MustExec("alter table t add index idx1(id, c1);") + tk.MustQuery("select * from t").Check(testkit.Rows("1 20200710010508")) + tk.MustExec("admin check table t") + // change timezone + tk.MustExec("set @@session.time_zone= default") + tk.MustExec("alter table t modify c1 timestamp") + // change timezone + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("set @@session.time_zone='SYSTEM'") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2020-07-10 01:05:08")) + tk.MustExec("admin check table t") + + // tests DST + // 1. modify a timestamp column to bigint + // 2. modify the bigint column to timestamp + tk.MustExec("drop table if exists t") + tk.MustExec("set @@session.time_zone=UTC") + tk.MustExec("create table t(id int primary key auto_increment, c1 timestamp default '1990-04-15 18:00:00');") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 1990-04-15 18:00:00")) + tk.MustExec("set @@session.time_zone='Asia/Shanghai'") + tk.MustExec("alter table t modify column c1 bigint;") + tk.MustQuery("select * from t").Check(testkit.Rows("1 19900416030000")) + tk.MustExec("alter table t modify c1 timestamp default '1990-04-15 18:00:00'") + tk.MustExec("set @@session.time_zone=UTC") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 1990-04-15 18:00:00", "2 1990-04-15 09:00:00")) + // 1. modify a timestamp column to bigint + // 2. add the index + // 3. modify the bigint column to timestamp + // The current session.time_zone is '+00:00'. + tk.MustExec("drop table if exists t") + tk.MustExec("set @@session.time_zone='-8:00'") + tk.MustExec("create table t(id int primary key auto_increment, c1 timestamp default '2016-03-13 02:30:00', index idx(c1));") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t").Check(testkit.Rows("1 2016-03-13 02:30:00")) + tk.MustExec("set @@session.time_zone='America/Los_Angeles'") + tk.MustExec("alter table t modify column c1 bigint;") + tk.MustQuery("select * from t").Check(testkit.Rows("1 20160313033000")) + tk.MustExec("alter table t add index idx1(id, c1);") + tk.MustExec("admin check table t") +} diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go index c1f591869e3b9..54860e5a3fec2 100644 --- a/ddl/db_cache_test.go +++ b/ddl/db_cache_test.go @@ -24,31 +24,19 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) func checkTableCacheStatus(t *testing.T, se session.Session, dbName, tableName string, status model.TableCacheStatusType) { - tb := testGetTableByNameT(t, se, dbName, tableName) + tb := testkit.TestGetTableByName(t, se, dbName, tableName) dom := domain.GetDomain(se) err := dom.Reload() require.NoError(t, err) require.Equal(t, status, tb.Meta().TableCacheStatusType) } -func testGetTableByNameT(t *testing.T, ctx sessionctx.Context, db, table string) table.Table { - dom := domain.GetDomain(ctx) - // Make sure the table schema is the new schema. - err := dom.Reload() - require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) - require.NoError(t, err) - return tbl -} - func TestAlterPartitionCache(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -240,10 +228,14 @@ func TestCacheTableSizeLimit(t *testing.T) { require.NoError(t, err) } + lastReadFromCache := func(tk *testkit.TestKit) bool { + return tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache + } + cached := false for i := 0; i < 200; i++ { tk.MustQuery("select count(*) from (select * from cache_t2 limit 1) t1").Check(testkit.Rows("1")) - if tk.HasPlan("select * from cache_t2", "UnionScan") { + if lastReadFromCache(tk) { cached = true break } diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 32af0c9b02da7..ef72ef45d81b5 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -554,6 +554,20 @@ func (s *testStateChangeSuite) TestWriteOnlyOnDupUpdateForAddColumns(c *C) { s.runTestInSchemaState(c, model.StateWriteOnly, true, addColumnsSQL, sqls, expectQuery) } +func (s *testStateChangeSuite) TestWriteReorgForModifyColumnTimestampToInt(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test_db_state") + tk.MustExec("drop table if exists tt") + tk.MustExec("create table tt(id int primary key auto_increment, c1 timestamp default '2020-07-10 01:05:08');") + tk.MustExec("insert into tt values();") + + sqls := make([]sqlWithErr, 1) + sqls[0] = sqlWithErr{"insert into tt values();", nil} + modifyColumnSQL := "alter table tt modify column c1 bigint;" + expectQuery := &expectQuery{"select c1 from tt", []string{"20200710010508", "20200710010508"}} + s.runTestInSchemaState(c, model.StateWriteReorganization, true, modifyColumnSQL, sqls, expectQuery) +} + type idxType byte const ( @@ -732,16 +746,19 @@ func (s *testStateChangeSuite) TestDeleteOnly(c *C) { s.runTestInSchemaState(c, model.StateDeleteOnly, true, dropColumnSQL, sqls, query) } -// TestDeleteOnlyForDropColumnWithIndexes test for delete data when a middle-state column with indexes in it. -func (s *testStateChangeSuite) TestDeleteOnlyForDropColumnWithIndexes(c *C) { +// TestSchemaChangeForDropColumnWithIndexes test for modify data when a middle-state column with indexes in it. +func (s *testStateChangeSuite) TestSchemaChangeForDropColumnWithIndexes(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db_state") - sqls := make([]sqlWithErr, 2) + sqls := make([]sqlWithErr, 5) sqls[0] = sqlWithErr{"delete from t1", nil} sqls[1] = sqlWithErr{"delete from t1 where b=1", errors.Errorf("[planner:1054]Unknown column 'b' in 'where clause'")} + sqls[2] = sqlWithErr{"insert into t1(a) values(1);", nil} + sqls[3] = sqlWithErr{"update t1 set a = 2 where a=1;", nil} + sqls[4] = sqlWithErr{"delete from t1", nil} prepare := func() { tk.MustExec("drop table if exists t1") - tk.MustExec("create table t1(a int key, b int, c int, index idx(b));") + tk.MustExec("create table t1(a bigint unsigned not null primary key, b int, c int, index idx(b));") tk.MustExec("insert into t1 values(1,1,1);") } prepare() @@ -754,6 +771,31 @@ func (s *testStateChangeSuite) TestDeleteOnlyForDropColumnWithIndexes(c *C) { s.runTestInSchemaState(c, model.StateDeleteReorganization, true, dropColumnSQL, sqls, query) } +// TestSchemaChangeForDropColumnWithIndexes test for modify data when some middle-state columns with indexes in it. +func (s *testStateChangeSuite) TestSchemaChangeForDropColumnsWithIndexes(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test_db_state") + sqls := make([]sqlWithErr, 5) + sqls[0] = sqlWithErr{"delete from t1", nil} + sqls[1] = sqlWithErr{"delete from t1 where b=1", errors.Errorf("[planner:1054]Unknown column 'b' in 'where clause'")} + sqls[2] = sqlWithErr{"insert into t1(a) values(1);", nil} + sqls[3] = sqlWithErr{"update t1 set a = 2 where a=1;", nil} + sqls[4] = sqlWithErr{"delete from t1", nil} + prepare := func() { + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(a bigint unsigned not null primary key, b int, c int, d int, index idx(b), index idx2(d));") + tk.MustExec("insert into t1 values(1,1,1,1);") + } + prepare() + dropColumnSQL := "alter table t1 drop column b, drop column d" + query := &expectQuery{sql: "select * from t1;", rows: []string{}} + s.runTestInSchemaState(c, model.StateWriteOnly, true, dropColumnSQL, sqls, query) + prepare() + s.runTestInSchemaState(c, model.StateDeleteOnly, true, dropColumnSQL, sqls, query) + prepare() + s.runTestInSchemaState(c, model.StateDeleteReorganization, true, dropColumnSQL, sqls, query) +} + // TestDeleteOnlyForDropExpressionIndex tests for deleting data when the hidden column is delete-only state. func (s *serialTestStateChangeSuite) TestDeleteOnlyForDropExpressionIndex(c *C) { _, err := s.se.Execute(context.Background(), "use test_db_state") diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index e88d5a62be8d3..966a90abc1845 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -22,9 +22,9 @@ import ( "strconv" "strings" "sync/atomic" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -40,99 +40,23 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" ) -var _ = Suite(&testIntegrationSuite1{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite2{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) -var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) - -type testIntegrationSuite struct { - lease time.Duration - cluster testutils.Cluster - store kv.Storage - dom *domain.Domain - ctx sessionctx.Context -} - -func setupIntegrationSuite(s *testIntegrationSuite, c *C) { - var err error - s.lease = 50 * time.Millisecond - ddl.SetWaitTimeWhenErrorOccurred(0) - - s.store, err = mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockstore.BootstrapWithSingleStore(c) - s.cluster = c - }), - ) - c.Assert(err, IsNil) - session.SetSchemaLease(s.lease) - session.DisableStats4Test() - s.dom, err = session.BootstrapSession(s.store) - c.Assert(err, IsNil) - - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - s.ctx = se.(sessionctx.Context) - _, err = se.Execute(context.Background(), "create database test_db") - c.Assert(err, IsNil) -} - -func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - -func tearDownIntegrationSuite(s *testIntegrationSuite, c *C) { - s.dom.Close() - s.store.Close() -} - -func (s *testIntegrationSuite) SetUpSuite(c *C) { - setupIntegrationSuite(s, c) -} - -func (s *testIntegrationSuite) TearDownSuite(c *C) { - tearDownIntegrationSuite(s, c) -} - -type testIntegrationSuite1 struct{ *testIntegrationSuite } -type testIntegrationSuite2 struct{ *testIntegrationSuite } - -func (s *testIntegrationSuite2) TearDownTest(c *C) { - tearDownIntegrationSuiteTest(s.testIntegrationSuite, c) -} - -type testIntegrationSuite3 struct{ *testIntegrationSuite } -type testIntegrationSuite4 struct{ *testIntegrationSuite } -type testIntegrationSuite5 struct{ *testIntegrationSuite } -type testIntegrationSuite6 struct{ *testIntegrationSuite } -type testIntegrationSuite7 struct{ *testIntegrationSuite } - -func (s *testIntegrationSuite5) TestNoZeroDateMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNoZeroDateMode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) defer tk.MustExec("set session sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION';") @@ -187,53 +111,53 @@ func (s *testIntegrationSuite5) TestNoZeroDateMode(c *C) { tk.MustGetErrCode(`alter table test_zero_date modify a date`, errno.ErrTruncatedWrongValue) } -func (s *testIntegrationSuite2) TestInvalidDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInvalidDefault(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") _, err := tk.Exec("create table t(c1 decimal default 1.7976931348623157E308)") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, types.ErrInvalidDefault), "err %v", err) _, err = tk.Exec("create table t( c1 varchar(2) default 'TiDB');") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, types.ErrInvalidDefault), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, types.ErrInvalidDefault), "err %v", err) } // TestKeyWithoutLength for issue #13452 -func (s testIntegrationSuite3) TestKeyWithoutLengthCreateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestKeyWithoutLengthCreateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test") _, err := tk.Exec("create table t_without_length (a text primary key)") - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, ".*BLOB/TEXT column 'a' used in key specification without a key length") + require.Error(t, err) + require.Regexp(t, ".*BLOB/TEXT column 'a' used in key specification without a key length", err.Error()) } // TestInvalidNameWhenCreateTable for issue #3848 -func (s *testIntegrationSuite3) TestInvalidNameWhenCreateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInvalidNameWhenCreateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") - _, err := tk.Exec("create table t(xxx.t.a bigint)") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue, Commentf("err %v", err)) - - _, err = tk.Exec("create table t(test.tttt.a bigint)") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongTableName), IsTrue, Commentf("err %v", err)) - - _, err = tk.Exec("create table t(t.tttt.a bigint)") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongDBName), IsTrue, Commentf("err %v", err)) + tk.MustGetErrCode("create table t(xxx.t.a bigint)", errno.ErrWrongDBName) + tk.MustGetErrCode("create table t(test.tttt.a bigint)", errno.ErrWrongTableName) + tk.MustGetErrCode("create table t(t.tttt.a bigint)", errno.ErrWrongDBName) } // TestCreateTableIfNotExists for issue #6879 -func (s *testIntegrationSuite3) TestCreateTableIfNotExists(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableIfNotExists(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") @@ -242,32 +166,36 @@ func (s *testIntegrationSuite3) TestCreateTableIfNotExists(c *C) { // Test duplicate create-table with `LIKE` clause tk.MustExec("create table if not exists ct like ct1;") - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(len(warnings), GreaterEqual, 1) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.GreaterOrEqual(t, len(warnings), 1) lastWarn := warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), IsTrue, Commentf("err %v", lastWarn.Err)) - c.Assert(lastWarn.Level, Equals, stmtctx.WarnLevelNote) + require.Truef(t, terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), "err %v", lastWarn.Err) + require.Equal(t, stmtctx.WarnLevelNote, lastWarn.Level) // Test duplicate create-table without `LIKE` clause tk.MustExec("create table if not exists ct(b bigint, c varchar(60));") - warnings = tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(len(warnings), GreaterEqual, 1) + warnings = tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.GreaterOrEqual(t, len(warnings), 1) lastWarn = warnings[len(warnings)-1] - c.Assert(terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err), IsTrue) + require.True(t, terror.ErrorEqual(infoschema.ErrTableExists, lastWarn.Err)) } // for issue #9910 -func (s *testIntegrationSuite2) TestCreateTableWithKeyWord(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithKeyWord(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") _, err := tk.Exec("create table t1(pump varchar(20), drainer varchar(20), node_id varchar(20), node_state varchar(20));") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testIntegrationSuite6) TestUniqueKeyNullValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUniqueKeyNullValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(255))") @@ -281,8 +209,10 @@ func (s *testIntegrationSuite6) TestUniqueKeyNullValue(c *C) { tk.MustExec("admin check index t b") } -func (s *testIntegrationSuite2) TestUniqueKeyNullValueClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUniqueKeyNullValueClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists unique_null_val;") tk.MustExec("create database unique_null_val;") @@ -297,16 +227,20 @@ func (s *testIntegrationSuite2) TestUniqueKeyNullValueClusterIndex(c *C) { } // TestModifyColumnAfterAddIndex Issue 5134 -func (s *testIntegrationSuite3) TestModifyColumnAfterAddIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestModifyColumnAfterAddIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table city (city VARCHAR(2) KEY);") tk.MustExec("alter table city change column city city varchar(50);") tk.MustExec(`insert into city values ("abc"), ("abd");`) } -func (s *testIntegrationSuite3) TestIssue2293(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue2293(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_issue_2293 (a int)") tk.MustGetErrCode("alter table t_issue_2293 add b int not null default 'a'", errno.ErrInvalidDefault) @@ -314,13 +248,15 @@ func (s *testIntegrationSuite3) TestIssue2293(c *C) { tk.MustQuery("select * from t_issue_2293").Check(testkit.Rows("1")) } -func (s *testIntegrationSuite2) TestIssue6101(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue6101(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t1 (quantity decimal(2) unsigned);") _, err := tk.Exec("insert into t1 values (500), (-500), (~0), (-1);") terr := errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.ErrWarnDataOutOfRange)) + require.Equal(t, errors.ErrCode(errno.ErrWarnDataOutOfRange), terr.Code()) tk.MustExec("drop table t1") tk.MustExec("set sql_mode=''") @@ -330,30 +266,34 @@ func (s *testIntegrationSuite2) TestIssue6101(c *C) { tk.MustExec("drop table t1") } -func (s *testIntegrationSuite2) TestIssue19229(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue19229(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE enumt (type enum('a', 'b') );") _, err := tk.Exec("insert into enumt values('xxx');") terr := errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.WarnDataTruncated)) + require.Equal(t, errors.ErrCode(errno.WarnDataTruncated), terr.Code()) _, err = tk.Exec("insert into enumt values(-1);") terr = errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.WarnDataTruncated)) + require.Equal(t, errors.ErrCode(errno.WarnDataTruncated), terr.Code()) tk.MustExec("drop table enumt") tk.MustExec("CREATE TABLE sett (type set('a', 'b') );") _, err = tk.Exec("insert into sett values('xxx');") terr = errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.WarnDataTruncated)) + require.Equal(t, errors.ErrCode(errno.WarnDataTruncated), terr.Code()) _, err = tk.Exec("insert into sett values(-1);") terr = errors.Cause(err).(*terror.Error) - c.Assert(terr.Code(), Equals, errors.ErrCode(errno.WarnDataTruncated)) + require.Equal(t, errors.ErrCode(errno.WarnDataTruncated), terr.Code()) tk.MustExec("drop table sett") } -func (s *testIntegrationSuite7) TestIndexLength(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexLength(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table idx_len(a int(0), b timestamp(0), c datetime(0), d time(0), f float(0), g decimal(0))") tk.MustExec("create index idx on idx_len(a)") @@ -380,8 +320,10 @@ func (s *testIntegrationSuite7) TestIndexLength(c *C) { tk.MustExec("drop table idx_len;") } -func (s *testIntegrationSuite3) TestIssue3833(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue3833(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table issue3833 (b char(0), c binary(0), d varchar(0))") tk.MustGetErrCode("create index idx on issue3833 (b)", errno.ErrWrongKeyColumn) @@ -395,8 +337,10 @@ func (s *testIntegrationSuite3) TestIssue3833(c *C) { tk.MustGetErrCode("create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(d))", errno.ErrWrongKeyColumn) } -func (s *testIntegrationSuite1) TestIssue2858And2717(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue2858And2717(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_issue_2858_bit (a bit(64) default b'0')") @@ -412,8 +356,10 @@ func (s *testIntegrationSuite1) TestIssue2858And2717(c *C) { tk.MustExec(`alter table t_issue_2858_hex alter column a set default 0x321`) } -func (s *testIntegrationSuite1) TestIssue4432(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue4432(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table tx (col bit(10) default 'a')") @@ -437,8 +383,10 @@ func (s *testIntegrationSuite1) TestIssue4432(c *C) { tk.MustExec("drop table tx") } -func (s *testIntegrationSuite7) TestIssue5092(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue5092(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_issue_5092 (a int)") @@ -515,9 +463,11 @@ func (s *testIntegrationSuite7) TestIssue5092(c *C) { tk.MustExec("drop table t_issue_5092") } -func (s *testIntegrationSuite5) TestErrnoErrorCode(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestErrnoErrorCode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") // create database sql := "create database aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa" @@ -687,8 +637,10 @@ func (s *testIntegrationSuite5) TestErrnoErrorCode(c *C) { tk.MustExec("set global tidb_enable_change_multi_schema = true") } -func (s *testIntegrationSuite3) TestTableDDLWithFloatType(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTableDDLWithFloatType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustGetErrCode("create table t (a decimal(1, 2))", errno.ErrMBiggerThanD) @@ -702,18 +654,17 @@ func (s *testIntegrationSuite3) TestTableDDLWithFloatType(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSuite1) TestTableDDLWithTimeType(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestTableDDLWithTimeType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustGetErrCode("create table t (a time(7))", errno.ErrTooBigPrecision) tk.MustGetErrCode("create table t (a datetime(7))", errno.ErrTooBigPrecision) tk.MustGetErrCode("create table t (a timestamp(7))", errno.ErrTooBigPrecision) _, err := tk.Exec("create table t (a time(-1))") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("create table t (a datetime)") tk.MustGetErrCode("alter table t add column b time(7)", errno.ErrTooBigPrecision) tk.MustGetErrCode("alter table t add column b datetime(7)", errno.ErrTooBigPrecision) @@ -728,21 +679,23 @@ func (s *testIntegrationSuite1) TestTableDDLWithTimeType(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSuite2) TestUpdateMultipleTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateMultipleTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database umt_db") tk.MustExec("use umt_db") tk.MustExec("create table t1 (c1 int, c2 int)") tk.MustExec("insert t1 values (1, 1), (2, 2)") tk.MustExec("create table t2 (c1 int, c2 int)") tk.MustExec("insert t2 values (1, 3), (2, 5)") - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() dom := domain.GetDomain(ctx) is := dom.InfoSchema() db, ok := is.SchemaByName(model.NewCIStr("umt_db")) - c.Assert(ok, IsTrue) + require.True(t, ok) t1Tbl, err := is.TableByName(model.NewCIStr("umt_db"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) t1Info := t1Tbl.Meta() // Add a new column in write only state. @@ -757,16 +710,16 @@ func (s *testIntegrationSuite2) TestUpdateMultipleTable(c *C) { } t1Info.Columns = append(t1Info.Columns, newColumn) - err = kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) _, err = m.GenSchemaVersion() - c.Assert(err, IsNil) - c.Assert(m.UpdateTable(db.ID, t1Info), IsNil) + require.NoError(t, err) + require.Nil(t, m.UpdateTable(db.ID, t1Info)) return nil }) - c.Assert(err, IsNil) + require.NoError(t, err) err = dom.Reload() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("update t1, t2 set t1.c1 = 8, t2.c2 = 10 where t1.c2 = t2.c1") tk.MustQuery("select * from t1").Check(testkit.Rows("8 1", "8 2")) @@ -774,23 +727,25 @@ func (s *testIntegrationSuite2) TestUpdateMultipleTable(c *C) { newColumn.State = model.StatePublic - err = kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) _, err = m.GenSchemaVersion() - c.Assert(err, IsNil) - c.Assert(m.UpdateTable(db.ID, t1Info), IsNil) + require.NoError(t, err) + require.Nil(t, m.UpdateTable(db.ID, t1Info)) return nil }) - c.Assert(err, IsNil) + require.NoError(t, err) err = dom.Reload() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery("select * from t1").Check(testkit.Rows("8 1 9", "8 2 9")) tk.MustExec("drop database umt_db") } -func (s *testIntegrationSuite2) TestNullGeneratedColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNullGeneratedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -807,8 +762,10 @@ func (s *testIntegrationSuite2) TestNullGeneratedColumn(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSuite2) TestDependedGeneratedColumnPrior2GeneratedColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDependedGeneratedColumnPrior2GeneratedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (" + @@ -831,34 +788,10 @@ func (s *testIntegrationSuite2) TestDependedGeneratedColumnPrior2GeneratedColumn tk.MustExec("alter table t add column(e int as (c+1))") } -func (s *testIntegrationSuite3) TestChangingCharsetToUtf8(c *C) { - tk := testkit.NewTestKit(c, s.store) - - tk.MustExec("use test") - tk.MustExec("create table t1(a varchar(20) charset utf8)") - tk.MustExec("insert into t1 values (?)", "t1_value") - tk.MustExec("alter table t1 collate uTf8mB4_uNiCoDe_Ci charset Utf8mB4 charset uTF8Mb4 collate UTF8MB4_BiN") - tk.MustExec("alter table t1 modify column a varchar(20) charset utf8mb4") - tk.MustQuery("select * from t1;").Check(testkit.Rows("t1_value")) - - tk.MustExec("create table t(a varchar(20) charset latin1)") - tk.MustExec("insert into t values (?)", "t_value") - - tk.MustExec("alter table t modify column a varchar(20) charset latin1") - tk.MustQuery("select * from t;").Check(testkit.Rows("t_value")) - - tk.MustGetErrCode("alter table t modify column a varchar(20) charset utf8", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify column a varchar(20) charset utf8mb4", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify column a varchar(20) charset utf8 collate utf8_bin", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify column a varchar(20) charset utf8mb4 collate utf8mb4_general_ci", errno.ErrUnsupportedDDLOperation) - - tk.MustGetErrCode("alter table t modify column a varchar(20) charset utf8mb4 collate utf8bin", errno.ErrUnknownCollation) - tk.MustGetErrCode("alter table t collate LATIN1_GENERAL_CI charset utf8 collate utf8_bin", errno.ErrConflictingDeclarations) - tk.MustGetErrCode("alter table t collate LATIN1_GENERAL_CI collate UTF8MB4_UNICODE_ci collate utf8_bin", errno.ErrCollationCharsetMismatch) -} - -func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestChangingTableCharset(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test") tk.MustExec("create table t(a char(10)) charset latin1 collate latin1_bin") @@ -883,13 +816,13 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) { tk.MustExec("create table t(a varchar(10)) charset utf8") tk.MustExec("alter table t convert to charset utf8mb4;") checkCharset := func(chs, coll string) { - tbl := testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Charset, Equals, chs) - c.Assert(tbl.Meta().Collate, Equals, coll) + tbl := testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.NotNil(t, tbl) + require.Equal(t, chs, tbl.Meta().Charset) + require.Equal(t, coll, tbl.Meta().Collate) for _, col := range tbl.Meta().Columns { - c.Assert(col.Charset, Equals, chs) - c.Assert(col.Collate, Equals, coll) + require.Equal(t, chs, col.Charset) + require.Equal(t, coll, col.Collate) } } checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) @@ -911,50 +844,50 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) { checkCharset(charset.CharsetUTF8MB4, "utf8mb4_general_ci") // Mock table info with charset is "". Old TiDB maybe create table with charset is "". - db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test")) - c.Assert(ok, IsTrue) - tbl := testGetTableByName(c, s.ctx, "test", "t") + db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test")) + require.True(t, ok) + tbl := testkit.TestGetTableByName(t, tk.Session(), "test", "t") tblInfo := tbl.Meta().Clone() tblInfo.Charset = "" tblInfo.Collate = "" updateTableInfo := func(tblInfo *model.TableInfo) { mockCtx := mock.NewContext() - mockCtx.Store = s.store + mockCtx.Store = store err := mockCtx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) txn, err := mockCtx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) mt := meta.NewMeta(txn) err = mt.UpdateTable(db.ID, tblInfo) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) } updateTableInfo(tblInfo) // check table charset is "" tk.MustExec("alter table t add column b varchar(10);") // load latest schema. - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Charset, Equals, "") - c.Assert(tbl.Meta().Collate, Equals, "") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.NotNil(t, tbl) + require.Equal(t, "", tbl.Meta().Charset) + require.Equal(t, "", tbl.Meta().Collate) // Test when table charset is "", this for compatibility. tk.MustExec("alter table t convert to charset utf8mb4;") checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) // Test when column charset is "". - tbl = testGetTableByName(c, s.ctx, "test", "t") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") tblInfo = tbl.Meta().Clone() tblInfo.Columns[0].Charset = "" tblInfo.Columns[0].Collate = "" updateTableInfo(tblInfo) // check table charset is "" tk.MustExec("alter table t drop column b;") // load latest schema. - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Columns[0].Charset, Equals, "") - c.Assert(tbl.Meta().Columns[0].Collate, Equals, "") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.NotNil(t, tbl) + require.Equal(t, "", tbl.Meta().Columns[0].Charset) + require.Equal(t, "", tbl.Meta().Columns[0].Collate) tk.MustExec("alter table t convert to charset utf8mb4;") checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4) @@ -970,40 +903,42 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) { tk.MustExec("drop table t") tk.MustExec("create table t(a varchar(5) charset utf8) charset utf8") tk.MustExec("alter table t charset utf8mb4") - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Charset, Equals, "utf8mb4") - c.Assert(tbl.Meta().Collate, Equals, "utf8mb4_bin") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.NotNil(t, tbl) + require.Equal(t, "utf8mb4", tbl.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tbl.Meta().Collate) for _, col := range tbl.Meta().Columns { // Column charset and collate should remain unchanged. - c.Assert(col.Charset, Equals, "utf8") - c.Assert(col.Collate, Equals, "utf8_bin") + require.Equal(t, "utf8", col.Charset) + require.Equal(t, "utf8_bin", col.Collate) } tk.MustExec("drop table t") tk.MustExec("create table t(a varchar(5) charset utf8 collate utf8_unicode_ci) charset utf8 collate utf8_unicode_ci") - tk.MustExec("alter table t collate utf8_danish_ci") - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl, NotNil) - c.Assert(tbl.Meta().Charset, Equals, "utf8") - c.Assert(tbl.Meta().Collate, Equals, "utf8_danish_ci") + tk.MustExec("alter table t collate utf8_general_ci") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.NotNil(t, tbl) + require.Equal(t, "utf8", tbl.Meta().Charset) + require.Equal(t, "utf8_general_ci", tbl.Meta().Collate) for _, col := range tbl.Meta().Columns { - c.Assert(col.Charset, Equals, "utf8") + require.Equal(t, "utf8", col.Charset) // Column collate should remain unchanged. - c.Assert(col.Collate, Equals, "utf8_unicode_ci") + require.Equal(t, "utf8_unicode_ci", col.Collate) } } -func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestModifyColumnOption(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") errMsg := "[ddl:8200]" // unsupported modify column with references assertErrCode := func(sql string, errCodeStr string) { _, err := tk.Exec(sql) - c.Assert(err, NotNil) - c.Assert(err.Error()[:len(errCodeStr)], Equals, errCodeStr) + require.Error(t, err) + require.Equal(t, errCodeStr, err.Error()[:len(errCodeStr)]) } tk.MustExec("drop table if exists t1") @@ -1020,19 +955,21 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { tk.MustExec("create table t2 (b char, c int)") assertErrCode("alter table t2 modify column c int references t1(a)", errMsg) _, err := tk.Exec("alter table t1 change a a varchar(16)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("alter table t1 change a a varchar(10)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("alter table t1 change a a datetime") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("alter table t1 change a a int(11) unsigned") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("alter table t2 change b b int(11) unsigned") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1046,8 +983,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn1(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1061,8 +1000,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1076,8 +1017,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn3(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1091,8 +1034,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn4(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1106,8 +1051,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexOnMultipleGeneratedColumn5(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -1124,8 +1071,10 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite6) TestCaseInsensitiveCharsetAndCollate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCaseInsensitiveCharsetAndCollate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_charset_collate") defer tk.MustExec("drop database test_charset_collate") @@ -1139,36 +1088,36 @@ func (s *testIntegrationSuite6) TestCaseInsensitiveCharsetAndCollate(c *C) { tk.MustExec("create table t5(a varchar(20)) ENGINE=InnoDB DEFAULT CHARSET=UTF8MB4 COLLATE=UTF8MB4_GENERAL_CI;") tk.MustExec("insert into t5 values ('特克斯和凯科斯群岛')") - db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test_charset_collate")) - c.Assert(ok, IsTrue) - tbl := testGetTableByName(c, s.ctx, "test_charset_collate", "t5") + db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test_charset_collate")) + require.True(t, ok) + tbl := testkit.TestGetTableByName(t, tk.Session(), "test_charset_collate", "t5") tblInfo := tbl.Meta().Clone() - c.Assert(tblInfo.Charset, Equals, "utf8mb4") - c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + require.Equal(t, "utf8mb4", tblInfo.Charset) + require.Equal(t, "utf8mb4", tblInfo.Columns[0].Charset) tblInfo.Version = model.TableInfoVersion2 tblInfo.Charset = "UTF8MB4" updateTableInfo := func(tblInfo *model.TableInfo) { mockCtx := mock.NewContext() - mockCtx.Store = s.store + mockCtx.Store = store err := mockCtx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) txn, err := mockCtx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) mt := meta.NewMeta(txn) - c.Assert(ok, IsTrue) + require.True(t, ok) err = mt.UpdateTable(db.ID, tblInfo) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) } updateTableInfo(tblInfo) tk.MustExec("alter table t5 add column b varchar(10);") // load latest schema. - tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() - c.Assert(tblInfo.Charset, Equals, "utf8mb4") - c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + tblInfo = testkit.TestGetTableByName(t, tk.Session(), "test_charset_collate", "t5").Meta() + require.Equal(t, "utf8mb4", tblInfo.Charset) + require.Equal(t, "utf8mb4", tblInfo.Columns[0].Charset) // For model.TableInfoVersion3, it is believed that all charsets / collations are lower-cased, do not do case-convert tblInfo = tblInfo.Clone() @@ -1177,19 +1126,21 @@ func (s *testIntegrationSuite6) TestCaseInsensitiveCharsetAndCollate(c *C) { updateTableInfo(tblInfo) tk.MustExec("alter table t5 add column c varchar(10);") // load latest schema. - tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() - c.Assert(tblInfo.Charset, Equals, "UTF8MB4") - c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + tblInfo = testkit.TestGetTableByName(t, tk.Session(), "test_charset_collate", "t5").Meta() + require.Equal(t, "UTF8MB4", tblInfo.Charset) + require.Equal(t, "utf8mb4", tblInfo.Columns[0].Charset) } -func (s *testIntegrationSuite3) TestZeroFillCreateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestZeroFillCreateTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists abc;") tk.MustExec("create table abc(y year, z tinyint(10) zerofill, primary key(y));") - is := s.dom.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("abc")) - c.Assert(err, IsNil) + require.NoError(t, err) var yearCol, zCol *model.ColumnInfo for _, col := range tbl.Meta().Columns { if col.Name.String() == "y" { @@ -1199,16 +1150,18 @@ func (s *testIntegrationSuite3) TestZeroFillCreateTable(c *C) { zCol = col } } - c.Assert(yearCol, NotNil) - c.Assert(yearCol.Tp, Equals, mysql.TypeYear) - c.Assert(mysql.HasUnsignedFlag(yearCol.Flag), IsTrue) + require.NotNil(t, yearCol) + require.Equal(t, mysql.TypeYear, yearCol.Tp) + require.True(t, mysql.HasUnsignedFlag(yearCol.Flag)) - c.Assert(zCol, NotNil) - c.Assert(mysql.HasUnsignedFlag(zCol.Flag), IsTrue) + require.NotNil(t, zCol) + require.True(t, mysql.HasUnsignedFlag(zCol.Flag)) } -func (s *testIntegrationSuite5) TestBitDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBitDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_bit (c1 bit(10) default 250, c2 int);") tk.MustExec("insert into t_bit set c2=1;") @@ -1277,8 +1230,14 @@ func (s *testIntegrationSuite5) TestBitDefaultValue(c *C) { );`) } -func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBackwardCompatibility(t *testing.T) { + var cluster testutils.Cluster + store, dom, clean := testkit.CreateMockStoreAndDomain(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockstore.BootstrapWithSingleStore(c) + cluster = c + })) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_backward_compatibility") defer tk.MustExec("drop database test_backward_compatibility") tk.MustExec("use test_backward_compatibility") @@ -1288,17 +1247,17 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { } // alter table t add index idx_b(b); - is := s.dom.InfoSchema() + is := dom.InfoSchema() schemaName := model.NewCIStr("test_backward_compatibility") tableName := model.NewCIStr("t") schema, ok := is.SchemaByName(schemaName) - c.Assert(ok, IsTrue) + require.True(t, ok) tbl, err := is.TableByName(schemaName, tableName) - c.Assert(err, IsNil) + require.NoError(t, err) // Split the table. tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) - s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) + cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) unique := false indexName := model.NewCIStr("idx_b") @@ -1319,30 +1278,30 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption}, } - txn, err := s.store.Begin() - c.Assert(err, IsNil) - t := meta.NewMeta(txn) - job.ID, err = t.GenGlobalID() - c.Assert(err, IsNil) + txn, err := store.Begin() + require.NoError(t, err) + tt := meta.NewMeta(txn) + job.ID, err = tt.GenGlobalID() + require.NoError(t, err) job.Version = 1 job.StartTS = txn.StartTS() // Simulate old TiDB init the add index job, old TiDB will not init the model.Job.ReorgMeta field, // if we set job.SnapshotVer here, can simulate the behavior. job.SnapshotVer = txn.StartTS() - err = t.EnQueueDDLJob(job) - c.Assert(err, IsNil) + err = tt.EnQueueDDLJob(job) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) - ticker := time.NewTicker(s.lease) + require.NoError(t, err) + ticker := time.NewTicker(50 * time.Millisecond) defer ticker.Stop() for range ticker.C { - historyJob, err := getHistoryDDLJob(s.store, job.ID) - c.Assert(err, IsNil) + historyJob, err := getHistoryDDLJob(store, job.ID) + require.NoError(t, err) if historyJob == nil { continue } - c.Assert(historyJob.Error, IsNil) + require.Nil(t, historyJob.Error) if historyJob.IsSynced() { break @@ -1353,38 +1312,6 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { tk.MustExec("admin check index t idx_b") } -type testMaxTableRowIDContext struct { - c *C - d ddl.DDL - tbl table.Table -} - -func newTestMaxTableRowIDContext(c *C, d ddl.DDL, tbl table.Table) *testMaxTableRowIDContext { - return &testMaxTableRowIDContext{ - c: c, - d: d, - tbl: tbl, - } -} - -func getMaxTableHandle(ctx *testMaxTableRowIDContext, store kv.Storage) (kv.Handle, bool) { - c := ctx.c - d := ctx.d - tbl := ctx.tbl - curVer, err := store.CurrentVersion(kv.GlobalTxnScope) - c.Assert(err, IsNil) - maxHandle, emptyTable, err := d.GetTableMaxHandle(curVer.Ver, tbl.(table.PhysicalTable)) - c.Assert(err, IsNil) - return maxHandle, emptyTable -} - -func checkGetMaxTableRowID(ctx *testMaxTableRowIDContext, store kv.Storage, expectEmpty bool, expectMaxHandle kv.Handle) { - c := ctx.c - maxHandle, emptyTable := getMaxTableHandle(ctx, store) - c.Assert(emptyTable, Equals, expectEmpty) - c.Assert(maxHandle, testutil.HandleEquals, expectMaxHandle) -} - func getHistoryDDLJob(store kv.Storage, id int64) (*model.Job, error) { var job *model.Job @@ -1398,11 +1325,10 @@ func getHistoryDDLJob(store kv.Storage, id int64) (*model.Job, error) { return job, errors.Trace(err) } -func (s *testIntegrationSuite6) TestCreateTableTooLarge(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableTooLarge(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") sql := "create table t_too_large (" @@ -1419,12 +1345,14 @@ func (s *testIntegrationSuite6) TestCreateTableTooLarge(c *C) { originLimit := config.GetGlobalConfig().TableColumnCountLimit atomic.StoreUint32(&config.GetGlobalConfig().TableColumnCountLimit, uint32(cnt*4)) _, err := tk.Exec(sql) - c.Assert(kv.ErrEntryTooLarge.Equal(err), IsTrue, Commentf("err:%v", err)) + require.Truef(t, kv.ErrEntryTooLarge.Equal(err), "err:%v", err) atomic.StoreUint32(&config.GetGlobalConfig().TableColumnCountLimit, originLimit) } -func (s *testSerialDBSuite1) TestCreateTableTooManyIndexes(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableTooManyIndexes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") sql := "create table t_too_many_indexes (" @@ -1443,8 +1371,10 @@ func (s *testSerialDBSuite1) TestCreateTableTooManyIndexes(c *C) { tk.MustGetErrCode(sql, errno.ErrTooManyKeys) } -func (s *testIntegrationSuite3) TestChangeColumnPosition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestChangeColumnPosition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table position (a int default 1, b int default 2)") @@ -1491,11 +1421,13 @@ func (s *testIntegrationSuite3) TestChangeColumnPosition(c *C) { " KEY `t` (`c`)", ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", } - c.Assert(createSQL, Equals, strings.Join(expectedSQL, "\n")) + require.Equal(t, strings.Join(expectedSQL, "\n"), createSQL) } -func (s *testIntegrationSuite2) TestAddIndexAfterAddColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddIndexAfterAddColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table test_add_index_after_add_col(a int, b int not null default '0')") @@ -1507,16 +1439,18 @@ func (s *testIntegrationSuite2) TestAddIndexAfterAddColumn(c *C) { tk.MustGetErrCode(sql, errno.ErrTooManyKeyParts) } -func (s *testIntegrationSuite3) TestResolveCharset(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestResolveCharset(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists resolve_charset") tk.MustExec(`CREATE TABLE resolve_charset (a varchar(255) DEFAULT NULL) DEFAULT CHARSET=latin1`) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("resolve_charset")) - c.Assert(err, IsNil) - c.Assert(tbl.Cols()[0].Charset, Equals, "latin1") + require.NoError(t, err) + require.Equal(t, "latin1", tbl.Cols()[0].Charset) tk.MustExec("INSERT INTO resolve_charset VALUES('鰈')") tk.MustExec("create database resolve_charset charset binary") @@ -1525,20 +1459,22 @@ func (s *testIntegrationSuite3) TestResolveCharset(c *C) { is = domain.GetDomain(ctx).InfoSchema() tbl, err = is.TableByName(model.NewCIStr("resolve_charset"), model.NewCIStr("resolve_charset")) - c.Assert(err, IsNil) - c.Assert(tbl.Cols()[0].Charset, Equals, "latin1") - c.Assert(tbl.Meta().Charset, Equals, "latin1") + require.NoError(t, err) + require.Equal(t, "latin1", tbl.Cols()[0].Charset) + require.Equal(t, "latin1", tbl.Meta().Charset) tk.MustExec(`CREATE TABLE resolve_charset1 (a varchar(255) DEFAULT NULL)`) is = domain.GetDomain(ctx).InfoSchema() tbl, err = is.TableByName(model.NewCIStr("resolve_charset"), model.NewCIStr("resolve_charset1")) - c.Assert(err, IsNil) - c.Assert(tbl.Cols()[0].Charset, Equals, "binary") - c.Assert(tbl.Meta().Charset, Equals, "binary") + require.NoError(t, err) + require.Equal(t, "binary", tbl.Cols()[0].Charset) + require.Equal(t, "binary", tbl.Meta().Charset) } -func (s *testIntegrationSuite6) TestAddColumnTooMany(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddColumnTooMany(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") count := int(atomic.LoadUint32(&config.GetGlobalConfig().TableColumnCountLimit) - 1) var cols []string @@ -1552,8 +1488,10 @@ func (s *testIntegrationSuite6) TestAddColumnTooMany(c *C) { tk.MustGetErrCode(alterSQL, errno.ErrTooManyFields) } -func (s *testSerialDBSuite1) TestCreateTooManyIndexes(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTooManyIndexes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") count := config.GetGlobalConfig().IndexLimit - 1 sql := "create table t_index_too_many (" @@ -1574,8 +1512,10 @@ func (s *testSerialDBSuite1) TestCreateTooManyIndexes(c *C) { tk.MustGetErrCode(alterSQL, errno.ErrTooManyKeys) } -func (s *testSerialDBSuite1) TestCreateSecondaryIndexInCluster(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateSecondaryIndexInCluster(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // test create table with non-unique key @@ -1655,51 +1595,53 @@ CREATE TABLE t ( tk.MustExec("create table t2 like t") } -func (s *testIntegrationSuite3) TestAlterColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestAlterColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create table test_alter_column (a int default 111, b varchar(8), c varchar(8) not null, d timestamp on update current_timestamp)") tk.MustExec("insert into test_alter_column set b = 'a', c = 'aa'") tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111")) - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session() is := domain.GetDomain(ctx).InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("test_alter_column")) - c.Assert(err, IsNil) + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_alter_column")) + require.NoError(t, err) tblInfo := tbl.Meta() colA := tblInfo.Columns[0] hasNoDefault := mysql.HasNoDefaultValueFlag(colA.Flag) - c.Assert(hasNoDefault, IsFalse) + require.False(t, hasNoDefault) tk.MustExec("alter table test_alter_column alter column a set default 222") tk.MustExec("insert into test_alter_column set b = 'b', c = 'bb'") tk.MustQuery("select a from test_alter_column").Check(testkit.Rows("111", "222")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("test_alter_column")) - c.Assert(err, IsNil) + tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_alter_column")) + require.NoError(t, err) tblInfo = tbl.Meta() colA = tblInfo.Columns[0] hasNoDefault = mysql.HasNoDefaultValueFlag(colA.Flag) - c.Assert(hasNoDefault, IsFalse) + require.False(t, hasNoDefault) tk.MustExec("alter table test_alter_column alter column b set default null") tk.MustExec("insert into test_alter_column set c = 'cc'") tk.MustQuery("select b from test_alter_column").Check(testkit.Rows("a", "b", "")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("test_alter_column")) - c.Assert(err, IsNil) + tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_alter_column")) + require.NoError(t, err) tblInfo = tbl.Meta() colC := tblInfo.Columns[2] hasNoDefault = mysql.HasNoDefaultValueFlag(colC.Flag) - c.Assert(hasNoDefault, IsTrue) + require.True(t, hasNoDefault) tk.MustExec("alter table test_alter_column alter column c set default 'xx'") tk.MustExec("insert into test_alter_column set a = 123") tk.MustQuery("select c from test_alter_column").Check(testkit.Rows("aa", "bb", "cc", "xx")) is = domain.GetDomain(ctx).InfoSchema() - tbl, err = is.TableByName(model.NewCIStr("test_db"), model.NewCIStr("test_alter_column")) - c.Assert(err, IsNil) + tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_alter_column")) + require.NoError(t, err) tblInfo = tbl.Meta() colC = tblInfo.Columns[2] hasNoDefault = mysql.HasNoDefaultValueFlag(colC.Flag) - c.Assert(hasNoDefault, IsFalse) + require.False(t, hasNoDefault) // TODO: After fix issue 2606. // tk.MustExec( "alter table test_alter_column alter column d set default null") tk.MustExec("alter table test_alter_column alter column a drop default") @@ -1721,13 +1663,13 @@ func (s *testIntegrationSuite3) TestAlterColumn(c *C) { tk.MustExec("drop table if exists mc") tk.MustExec("create table mc(a int key nonclustered, b int, c int)") _, err = tk.Exec("alter table mc modify column a int key") // Adds a new primary key - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("alter table mc modify column c int unique") // Adds a new unique key - c.Assert(err, NotNil) + require.Error(t, err) result := tk.MustQuery("show create table mc") createSQL := result.Rows()[0][1] expected := "CREATE TABLE `mc` (\n `a` int(11) NOT NULL,\n `b` int(11) DEFAULT NULL,\n `c` int(11) DEFAULT NULL,\n PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" - c.Assert(createSQL, Equals, expected) + require.Equal(t, expected, createSQL) // Change / modify column should preserve index options. tk.MustExec("drop table if exists mc") @@ -1738,7 +1680,7 @@ func (s *testIntegrationSuite3) TestAlterColumn(c *C) { result = tk.MustQuery("show create table mc") createSQL = result.Rows()[0][1] expected = "CREATE TABLE `mc` (\n `a` bigint(20) NOT NULL,\n `b` bigint(20) DEFAULT NULL,\n `c` bigint(20) DEFAULT NULL,\n PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */,\n UNIQUE KEY `c` (`c`)\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" - c.Assert(createSQL, Equals, expected) + require.Equal(t, expected, createSQL) // Dropping or keeping auto_increment is allowed, however adding is not allowed. tk.MustExec("drop table if exists mc") @@ -1747,18 +1689,18 @@ func (s *testIntegrationSuite3) TestAlterColumn(c *C) { result = tk.MustQuery("show create table mc") createSQL = result.Rows()[0][1] expected = "CREATE TABLE `mc` (\n `a` bigint(20) NOT NULL AUTO_INCREMENT,\n `b` int(11) DEFAULT NULL,\n PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" - c.Assert(createSQL, Equals, expected) + require.Equal(t, expected, createSQL) _, err = tk.Exec("alter table mc modify column a bigint") // Droppping auto_increment is not allow when @@tidb_allow_remove_auto_inc == 'off' - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("set @@tidb_allow_remove_auto_inc = on") tk.MustExec("alter table mc modify column a bigint") // Dropping auto_increment is ok when @@tidb_allow_remove_auto_inc == 'on' result = tk.MustQuery("show create table mc") createSQL = result.Rows()[0][1] expected = "CREATE TABLE `mc` (\n `a` bigint(20) NOT NULL,\n `b` int(11) DEFAULT NULL,\n PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" - c.Assert(createSQL, Equals, expected) + require.Equal(t, expected, createSQL) _, err = tk.Exec("alter table mc modify column a bigint auto_increment") // Adds auto_increment should throw error - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("drop table if exists t") tk.MustExec("create table t1 (a varchar(10),b varchar(100),c tinyint,d varchar(3071),index(a),index(a,b),index (c,d)) charset = ascii;") @@ -1785,31 +1727,29 @@ func (s *testIntegrationSuite3) TestAlterColumn(c *C) { tk.MustExec("alter table t1 add column update_time3 datetime(3) NOT NULL DEFAULT CURRENT_TIMESTAMP(3);") tk.MustExec("alter table t1 add column update_time6 datetime(6) NOT NULL DEFAULT CURRENT_TIMESTAMP(6);") rows := tk.MustQuery("select * from t1").Rows() - c.Assert(rows[0][0], Equals, "1") + require.Equal(t, "1", rows[0][0]) updateTime3 := rows[0][1].(string) - c.Assert(updateTime3[len(updateTime3)-3:], Not(Equals), "000") + require.NotEqual(t, "000", updateTime3[len(updateTime3)-3:]) updateTime6 := rows[0][2].(string) - c.Assert(updateTime6[len(updateTime6)-6:], Not(Equals), "000000") + require.NotEqual(t, "000000", updateTime6[len(updateTime6)-6:]) } -func (s *testIntegrationSuite) assertWarningExec(tk *testkit.TestKit, c *C, sql string, expectedWarn *terror.Error) { +func assertWarningExec(tk *testkit.TestKit, t *testing.T, sql string, expectedWarn *terror.Error) { _, err := tk.Exec(sql) - c.Assert(err, IsNil) - st := tk.Se.GetSessionVars().StmtCtx - c.Assert(st.WarningCount(), Equals, uint16(1)) - c.Assert(expectedWarn.Equal(st.GetWarnings()[0].Err), IsTrue, Commentf("error:%v", err)) -} - -func (s *testIntegrationSuite) assertAlterWarnExec(tk *testkit.TestKit, c *C, sql string) { - s.assertWarningExec(tk, c, sql, ddl.ErrAlterOperationNotSupported) + require.NoError(t, err) + st := tk.Session().GetSessionVars().StmtCtx + require.Equal(t, uint16(1), st.WarningCount()) + require.Truef(t, expectedWarn.Equal(st.GetWarnings()[0].Err), "error:%v", err) } -func (s *testIntegrationSuite) assertAlterErrorExec(tk *testkit.TestKit, c *C, sql string) { - tk.MustGetErrCode(sql, errno.ErrAlterOperationNotSupportedReason) +func assertAlterWarnExec(tk *testkit.TestKit, t *testing.T, sql string) { + assertWarningExec(tk, t, sql, ddl.ErrAlterOperationNotSupported) } -func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterAlgorithm(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, t1") defer tk.MustExec("drop table if exists t") @@ -1824,57 +1764,59 @@ func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) { PARTITION p2 VALUES LESS THAN (16), PARTITION p3 VALUES LESS THAN (21) )`) - s.assertAlterWarnExec(tk, c, "alter table t modify column a bigint, ALGORITHM=INPLACE;") + assertAlterWarnExec(tk, t, "alter table t modify column a bigint, ALGORITHM=INPLACE;") tk.MustExec("alter table t modify column a bigint, ALGORITHM=INPLACE, ALGORITHM=INSTANT;") tk.MustExec("alter table t modify column a bigint, ALGORITHM=DEFAULT;") // Test add/drop index - s.assertAlterErrorExec(tk, c, "alter table t add index idx_b(b), ALGORITHM=INSTANT") - s.assertAlterWarnExec(tk, c, "alter table t add index idx_b1(b), ALGORITHM=COPY") + tk.MustGetErrCode("alter table t add index idx_b(b), ALGORITHM=INSTANT", errno.ErrAlterOperationNotSupportedReason) + assertAlterWarnExec(tk, t, "alter table t add index idx_b1(b), ALGORITHM=COPY") tk.MustExec("alter table t add index idx_b2(b), ALGORITHM=INPLACE") tk.MustExec("alter table t add index idx_b3(b), ALGORITHM=DEFAULT") - s.assertAlterWarnExec(tk, c, "alter table t drop index idx_b3, ALGORITHM=INPLACE") - s.assertAlterWarnExec(tk, c, "alter table t drop index idx_b1, ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t drop index idx_b3, ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t drop index idx_b1, ALGORITHM=COPY") tk.MustExec("alter table t drop index idx_b2, ALGORITHM=INSTANT") // Test rename - s.assertAlterWarnExec(tk, c, "alter table t rename to t1, ALGORITHM=COPY") - s.assertAlterWarnExec(tk, c, "alter table t1 rename to t2, ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t rename to t1, ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t1 rename to t2, ALGORITHM=INPLACE") tk.MustExec("alter table t2 rename to t, ALGORITHM=INSTANT") tk.MustExec("alter table t rename to t1, ALGORITHM=DEFAULT") tk.MustExec("alter table t1 rename to t") // Test rename index - s.assertAlterWarnExec(tk, c, "alter table t rename index idx_c to idx_c1, ALGORITHM=COPY") - s.assertAlterWarnExec(tk, c, "alter table t rename index idx_c1 to idx_c2, ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t rename index idx_c to idx_c1, ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t rename index idx_c1 to idx_c2, ALGORITHM=INPLACE") tk.MustExec("alter table t rename index idx_c2 to idx_c, ALGORITHM=INSTANT") tk.MustExec("alter table t rename index idx_c to idx_c1, ALGORITHM=DEFAULT") // partition. - s.assertAlterWarnExec(tk, c, "alter table t ALGORITHM=COPY, truncate partition p1") - s.assertAlterWarnExec(tk, c, "alter table t ALGORITHM=INPLACE, truncate partition p2") + assertAlterWarnExec(tk, t, "alter table t ALGORITHM=COPY, truncate partition p1") + assertAlterWarnExec(tk, t, "alter table t ALGORITHM=INPLACE, truncate partition p2") tk.MustExec("alter table t ALGORITHM=INSTANT, truncate partition p3") - s.assertAlterWarnExec(tk, c, "alter table t add partition (partition p4 values less than (2002)), ALGORITHM=COPY") - s.assertAlterWarnExec(tk, c, "alter table t add partition (partition p5 values less than (3002)), ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t add partition (partition p4 values less than (2002)), ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t add partition (partition p5 values less than (3002)), ALGORITHM=INPLACE") tk.MustExec("alter table t add partition (partition p6 values less than (4002)), ALGORITHM=INSTANT") - s.assertAlterWarnExec(tk, c, "alter table t ALGORITHM=COPY, drop partition p4") - s.assertAlterWarnExec(tk, c, "alter table t ALGORITHM=INPLACE, drop partition p5") + assertAlterWarnExec(tk, t, "alter table t ALGORITHM=COPY, drop partition p4") + assertAlterWarnExec(tk, t, "alter table t ALGORITHM=INPLACE, drop partition p5") tk.MustExec("alter table t ALGORITHM=INSTANT, drop partition p6") // Table options - s.assertAlterWarnExec(tk, c, "alter table t comment = 'test', ALGORITHM=COPY") - s.assertAlterWarnExec(tk, c, "alter table t comment = 'test', ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t comment = 'test', ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t comment = 'test', ALGORITHM=INPLACE") tk.MustExec("alter table t comment = 'test', ALGORITHM=INSTANT") - s.assertAlterWarnExec(tk, c, "alter table t default charset = utf8mb4, ALGORITHM=COPY") - s.assertAlterWarnExec(tk, c, "alter table t default charset = utf8mb4, ALGORITHM=INPLACE") + assertAlterWarnExec(tk, t, "alter table t default charset = utf8mb4, ALGORITHM=COPY") + assertAlterWarnExec(tk, t, "alter table t default charset = utf8mb4, ALGORITHM=INPLACE") tk.MustExec("alter table t default charset = utf8mb4, ALGORITHM=INSTANT") } -func (s *testIntegrationSuite3) TestAlterTableAddUniqueOnPartionRangeColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddUniqueOnPartionRangeColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -1901,27 +1843,31 @@ func (s *testIntegrationSuite3) TestAlterTableAddUniqueOnPartionRangeColumn(c *C tk.MustGetErrCode("alter table t add unique index idx_b(b)", errno.ErrUniqueKeyNeedAllFieldsInPf) } -func (s *testIntegrationSuite5) TestFulltextIndexIgnore(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestFulltextIndexIgnore(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t_ft") defer tk.MustExec("drop table if exists t_ft") // Make sure that creating and altering to add a fulltext key gives the correct warning - s.assertWarningExec(tk, c, "create table t_ft (a text, fulltext key (a))", ddl.ErrTableCantHandleFt) - s.assertWarningExec(tk, c, "alter table t_ft add fulltext key (a)", ddl.ErrTableCantHandleFt) + assertWarningExec(tk, t, "create table t_ft (a text, fulltext key (a))", ddl.ErrTableCantHandleFt) + assertWarningExec(tk, t, "alter table t_ft add fulltext key (a)", ddl.ErrTableCantHandleFt) // Make sure table t_ft still has no indexes even after it was created and altered r := tk.MustQuery("show index from t_ft") - c.Assert(r.Rows(), HasLen, 0) + require.Len(t, r.Rows(), 0) r = tk.MustQuery("select * from information_schema.statistics where table_schema='test' and table_name='t_ft'") - c.Assert(r.Rows(), HasLen, 0) + require.Len(t, r.Rows(), 0) + } -func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestTreatOldVersionUTF8AsUTF8MB4(t *testing.T) { + restoreFunc := config.RestoreFunc() + defer restoreFunc() + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -1934,28 +1880,28 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // Mock old version table info with column charset is utf8. - db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test")) - tbl := testGetTableByName(c, s.ctx, "test", "t") + db, ok := domain.GetDomain(tk.Session()).InfoSchema().SchemaByName(model.NewCIStr("test")) + tbl := testkit.TestGetTableByName(t, tk.Session(), "test", "t") tblInfo := tbl.Meta().Clone() tblInfo.Version = model.TableInfoVersion0 tblInfo.Columns[0].Version = model.ColumnInfoVersion0 updateTableInfo := func(tblInfo *model.TableInfo) { mockCtx := mock.NewContext() - mockCtx.Store = s.store + mockCtx.Store = store err := mockCtx.NewTxn(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) txn, err := mockCtx.Txn(true) - c.Assert(err, IsNil) + require.NoError(t, err) mt := meta.NewMeta(txn) - c.Assert(ok, IsTrue) + require.True(t, ok) err = mt.UpdateTable(db.ID, tblInfo) - c.Assert(err, IsNil) + require.NoError(t, err) err = txn.Commit(context.Background()) - c.Assert(err, IsNil) + require.NoError(t, err) } updateTableInfo(tblInfo) tk.MustExec("alter table t add column c varchar(10) character set utf8;") // load latest schema. - c.Assert(config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4, IsTrue) + require.True(t, config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4) tk.MustExec("insert into t set a= x'f09f8c80'") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + " `a` varchar(10) DEFAULT NULL,\n" + @@ -1973,7 +1919,7 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // Mock old version table info with table and column charset is utf8. - tbl = testGetTableByName(c, s.ctx, "test", "t") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") tblInfo = tbl.Meta().Clone() tblInfo.Charset = charset.CharsetUTF8 tblInfo.Collate = charset.CollationUTF8 @@ -2007,10 +1953,10 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { conf.TreatOldVersionUTF8AsUTF8MB4 = true }) tk.MustExec("alter table t modify column a varchar(10) character set utf8mb4") // change column charset. - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl.Meta().Columns[0].Charset, Equals, charset.CharsetUTF8MB4) - c.Assert(tbl.Meta().Columns[0].Collate, Equals, charset.CollationUTF8MB4) - c.Assert(tbl.Meta().Columns[0].Version, Equals, model.ColumnInfoVersion0) + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.Equal(t, charset.CharsetUTF8MB4, tbl.Meta().Columns[0].Charset) + require.Equal(t, charset.CollationUTF8MB4, tbl.Meta().Columns[0].Collate) + require.Equal(t, model.ColumnInfoVersion0, tbl.Meta().Columns[0].Version) tk.MustExec("insert into t set a= x'f09f8c80'") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + " `a` varchar(10) DEFAULT NULL,\n" + @@ -2018,13 +1964,13 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) // Test for change column should not modify the column version. tk.MustExec("alter table t change column a a varchar(20)") // change column. - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl.Meta().Columns[0].Charset, Equals, charset.CharsetUTF8MB4) - c.Assert(tbl.Meta().Columns[0].Collate, Equals, charset.CollationUTF8MB4) - c.Assert(tbl.Meta().Columns[0].Version, Equals, model.ColumnInfoVersion0) + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.Equal(t, charset.CharsetUTF8MB4, tbl.Meta().Columns[0].Charset) + require.Equal(t, charset.CollationUTF8MB4, tbl.Meta().Columns[0].Collate) + require.Equal(t, model.ColumnInfoVersion0, tbl.Meta().Columns[0].Version) // Test for v2.1.5 and v2.1.6 that table version is 1 but column version is 0. - tbl = testGetTableByName(c, s.ctx, "test", "t") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") tblInfo = tbl.Meta().Clone() tblInfo.Charset = charset.CharsetUTF8 tblInfo.Collate = charset.CollationUTF8 @@ -2033,7 +1979,7 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { tblInfo.Columns[0].Charset = charset.CharsetUTF8 tblInfo.Columns[0].Collate = charset.CollationUTF8 updateTableInfo(tblInfo) - c.Assert(config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4, IsTrue) + require.True(t, config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4) tk.MustExec("alter table t change column b b varchar(20) character set ascii") // reload schema. tk.MustExec("insert into t set a= x'f09f8c80'") tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" + @@ -2068,18 +2014,22 @@ func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) } -func (s *testIntegrationSuite3) TestDefaultValueIsString(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDefaultValueIsString(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int default b'1');") - tbl := testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl.Meta().Columns[0].DefaultValue, Equals, "1") + tbl := testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.Equal(t, "1", tbl.Meta().Columns[0].DefaultValue) } -func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestChangingDBCharset(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("DROP DATABASE IF EXISTS alterdb1") tk.MustExec("CREATE DATABASE alterdb1 CHARSET=utf8 COLLATE=utf8_unicode_ci") @@ -2099,13 +2049,13 @@ func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) { }, } for _, fc := range noDBFailedCases { - c.Assert(tk.ExecToErr(fc.stmt).Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt)) + require.EqualError(t, tk.ExecToErr(fc.stmt), fc.errMsg) } verifyDBCharsetAndCollate := func(dbName, chs string, coll string) { // check `SHOW CREATE SCHEMA`. r := tk.MustQuery("SHOW CREATE SCHEMA " + dbName).Rows()[0][1].(string) - c.Assert(strings.Contains(r, "CHARACTER SET "+chs), IsTrue) + require.True(t, strings.Contains(r, "CHARACTER SET "+chs)) template := `SELECT DEFAULT_CHARACTER_SET_NAME, @@ -2115,14 +2065,14 @@ func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) { sql := fmt.Sprintf(template, dbName) tk.MustQuery(sql).Check(testkit.Rows(fmt.Sprintf("%s %s", chs, coll))) - dom := domain.GetDomain(s.ctx) + dom := domain.GetDomain(tk.Session()) // Make sure the table schema is the new schema. err := dom.Reload() - c.Assert(err, IsNil) + require.NoError(t, err) dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(dbName)) - c.Assert(ok, Equals, true) - c.Assert(dbInfo.Charset, Equals, chs) - c.Assert(dbInfo.Collate, Equals, coll) + require.Equal(t, true, ok) + require.Equal(t, chs, dbInfo.Charset) + require.Equal(t, coll, dbInfo.Collate) } tk.MustExec("ALTER SCHEMA alterdb1 COLLATE = utf8mb4_general_ci") @@ -2171,7 +2121,7 @@ func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) { } for _, fc := range failedCases { - c.Assert(tk.ExecToErr(fc.stmt).Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt)) + require.EqualError(t, tk.ExecToErr(fc.stmt), fc.errMsg) } tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8' COLLATE = 'utf8_unicode_ci'") verifyDBCharsetAndCollate("alterdb2", "utf8", "utf8_unicode_ci") @@ -2190,8 +2140,10 @@ func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) { tk.MustExec("alter database TEST_UPPERCASE_DB_CHARSET default character set utf8mb4;") } -func (s *testIntegrationSuite4) TestDropAutoIncrementIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropAutoIncrementIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -2206,8 +2158,10 @@ func (s *testIntegrationSuite4) TestDropAutoIncrementIndex(c *C) { tk.MustGetErrCode(dropIndexSQL, errno.ErrWrongAutoKey) } -func (s *testIntegrationSuite4) TestInsertIntoGeneratedColumnWithDefaultExpr(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertIntoGeneratedColumnWithDefaultExpr(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") @@ -2264,8 +2218,10 @@ func (s *testIntegrationSuite4) TestInsertIntoGeneratedColumnWithDefaultExpr(c * tk.MustExec("drop table t1, t2, t3, t4, t5") } -func (s *testIntegrationSuite3) TestSqlFunctionsInGeneratedColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSqlFunctionsInGeneratedColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test") tk.MustExec("use test") tk.MustExec("drop table if exists t, t1") @@ -2306,19 +2262,31 @@ func (s *testIntegrationSuite3) TestSqlFunctionsInGeneratedColumns(c *C) { tk.MustExec("create table t (a int, b int as ((a)))") } -func (s *testIntegrationSuite3) TestParserIssue284(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestParserIssue284(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table test.t_parser_issue_284(c1 int not null primary key)") _, err := tk.Exec("create table test.t_parser_issue_284_2(id int not null primary key, c1 int not null, constraint foreign key (c1) references t_parser_issue_284(c1))") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("drop table test.t_parser_issue_284") tk.MustExec("drop table test.t_parser_issue_284_2") } -func (s *testSerialDBSuite1) TestAddExpressionIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddExpressionIndex(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Log.SlowThreshold = 10000 + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") @@ -2327,37 +2295,37 @@ func (s *testSerialDBSuite1) TestAddExpressionIndex(c *C) { tk.MustExec("alter table t add index idx((a+b));") tk.MustQuery("SELECT * FROM INFORMATION_SCHEMA.KEY_COLUMN_USAGE WHERE table_name = 't'").Check(testkit.Rows()) - tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) columns := tblInfo.Meta().Columns - c.Assert(len(columns), Equals, 3) - c.Assert(columns[2].Hidden, IsTrue) + require.Equal(t, 3, len(columns)) + require.True(t, columns[2].Hidden) tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t add index idx_multi((a+b),(a+1), b);") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + tblInfo, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) columns = tblInfo.Meta().Columns - c.Assert(len(columns), Equals, 5) - c.Assert(columns[3].Hidden, IsTrue) - c.Assert(columns[4].Hidden, IsTrue) + require.Equal(t, 5, len(columns)) + require.True(t, columns[3].Hidden) + require.True(t, columns[4].Hidden) tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t drop index idx;") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + tblInfo, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) columns = tblInfo.Meta().Columns - c.Assert(len(columns), Equals, 4) + require.Equal(t, 4, len(columns)) tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) tk.MustExec("alter table t drop index idx_multi;") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + tblInfo, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) columns = tblInfo.Meta().Columns - c.Assert(len(columns), Equals, 2) + require.Equal(t, 2, len(columns)) tk.MustQuery("select * from t;").Check(testkit.Rows("1 2.1")) @@ -2390,8 +2358,18 @@ func (s *testSerialDBSuite1) TestAddExpressionIndex(c *C) { }) } -func (s *testSerialDBSuite1) TestCreateExpressionIndexError(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateExpressionIndexError(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Log.SlowThreshold = 10000 + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b real);") @@ -2435,8 +2413,10 @@ func (s *testSerialDBSuite1) TestCreateExpressionIndexError(c *C) { tk.MustExec("update t t1 set t1.short_name='a' where t1.id='1';") } -func (s *testSerialDBSuite1) TestAddExpressionIndexOnPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddExpressionIndexOnPartition(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec(`create table t( @@ -2452,11 +2432,11 @@ func (s *testSerialDBSuite1) TestAddExpressionIndexOnPartition(c *C) { tk.MustExec("insert into t values (1, 'test', 2), (12, 'test', 3), (15, 'test', 10), (20, 'test', 20);") tk.MustExec("alter table t add index idx((a+c));") - tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) columns := tblInfo.Meta().Columns - c.Assert(len(columns), Equals, 4) - c.Assert(columns[3].Hidden, IsTrue) + require.Equal(t, 4, len(columns)) + require.True(t, columns[3].Hidden) tk.MustQuery("select * from t order by a;").Check(testkit.Rows("1 test 2", "12 test 3", "15 test 10", "20 test 20")) } @@ -2464,17 +2444,19 @@ func (s *testSerialDBSuite1) TestAddExpressionIndexOnPartition(c *C) { // TestCreateTableWithAutoIdCache test the auto_id_cache table option. // `auto_id_cache` take effects on handle too when `PKIshandle` is false, // or even there is no auto_increment column at all. -func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTableWithAutoIdCache(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") // Test primary key is handle. tk.MustExec("create table t(a int auto_increment key clustered) auto_id_cache 100") - tblInfo, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(100)) + tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, int64(100), tblInfo.Meta().AutoIdCache) tk.MustExec("insert into t values()") tk.MustQuery("select * from t").Check(testkit.Rows("1")) tk.MustExec("delete from t") @@ -2488,8 +2470,8 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(a int) auto_id_cache 100") - _, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + _, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) tk.MustExec("insert into t values()") tk.MustQuery("select _tidb_rowid from t").Check(testkit.Rows("1")) @@ -2504,8 +2486,8 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(a int null, b int auto_increment unique) auto_id_cache 100") - _, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + _, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) tk.MustExec("insert into t(b) values(NULL)") tk.MustQuery("select b, _tidb_rowid from t").Check(testkit.Rows("1 2")) @@ -2519,9 +2501,9 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { // Test alter auto_id_cache. tk.MustExec("alter table t1 auto_id_cache 200") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(200)) + tblInfo, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, int64(200), tblInfo.Meta().AutoIdCache) tk.MustExec("insert into t1(b) values(NULL)") tk.MustQuery("select b, _tidb_rowid from t1").Check(testkit.Rows("201 202")) @@ -2536,9 +2518,9 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(a int auto_increment key clustered) auto_id_cache 3") - tblInfo, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) - c.Assert(tblInfo.Meta().AutoIdCache, Equals, int64(3)) + tblInfo, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + require.Equal(t, int64(3), tblInfo.Meta().AutoIdCache) // Test insert batch size(4 here) greater than the customized autoid step(3 here). tk.MustExec("insert into t(a) values(NULL),(NULL),(NULL)") @@ -2552,23 +2534,25 @@ func (s *testIntegrationSuite3) TestCreateTableWithAutoIdCache(c *C) { tk.MustExec("insert into t1(a) values(NULL)") next := tk.MustQuery("select a from t1").Rows()[0][0].(string) nextInt, err := strconv.Atoi(next) - c.Assert(err, IsNil) - c.Assert(nextInt, Greater, 5) + require.NoError(t, err) + require.Greater(t, nextInt, 5) // Test auto_id_cache overflows int64. tk.MustExec("drop table if exists t;") _, err = tk.Exec("create table t(a int) auto_id_cache = 9223372036854775808") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64") + require.Error(t, err) + require.Equal(t, "table option auto_id_cache overflows int64", err.Error()) tk.MustExec("create table t(a int) auto_id_cache = 9223372036854775807") _, err = tk.Exec("alter table t auto_id_cache = 9223372036854775808") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "table option auto_id_cache overflows int64") + require.Error(t, err) + require.Equal(t, "table option auto_id_cache overflows int64", err.Error()) } -func (s *testIntegrationSuite4) TestAlterIndexVisibility(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterIndexVisibility(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists alter_index_test") tk.MustExec("USE alter_index_test;") tk.MustExec("drop table if exists t, t1, t2, t3;") @@ -2610,8 +2594,10 @@ func queryIndexOnTable(dbName, tableName string) string { return fmt.Sprintf("select distinct index_name, is_visible from information_schema.statistics where table_schema = '%s' and table_name = '%s' order by index_name", dbName, tableName) } -func (s *testIntegrationSuite5) TestDropColumnWithCompositeIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropColumnWithCompositeIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) query := queryIndexOnTable("drop_composite_index_test", "t_drop_column_with_comp_idx") tk.MustExec("create database if not exists drop_composite_index_test") tk.MustExec("use drop_composite_index_test") @@ -2627,60 +2613,70 @@ func (s *testIntegrationSuite5) TestDropColumnWithCompositeIndex(c *C) { tk.MustQuery(query).Check(testkit.Rows("idx_b NO", "idx_bc NO")) } -func (s *testIntegrationSuite5) TestDropColumnWithIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestDropColumnWithIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create table t_drop_column_with_idx(a int, b int, c int)") defer tk.MustExec("drop table if exists t_drop_column_with_idx") tk.MustExec("create index idx on t_drop_column_with_idx(b)") tk.MustExec("alter table t_drop_column_with_idx drop column b") - query := queryIndexOnTable("test_db", "t_drop_column_with_idx") + query := queryIndexOnTable("test", "t_drop_column_with_idx") tk.MustQuery(query).Check(testkit.Rows()) } -func (s *testIntegrationSuite5) TestDropColumnWithMultiIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestDropColumnWithMultiIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create table t_drop_column_with_idx(a int, b int, c int)") defer tk.MustExec("drop table if exists t_drop_column_with_idx") tk.MustExec("create index idx_1 on t_drop_column_with_idx(b)") tk.MustExec("create index idx_2 on t_drop_column_with_idx(b)") tk.MustExec("alter table t_drop_column_with_idx drop column b") - query := queryIndexOnTable("test_db", "t_drop_column_with_idx") + query := queryIndexOnTable("test", "t_drop_column_with_idx") tk.MustQuery(query).Check(testkit.Rows()) } -func (s *testIntegrationSuite5) TestDropColumnsWithMultiIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestDropColumnsWithMultiIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create table t_drop_columns_with_idx(a int, b int, c int)") defer tk.MustExec("drop table if exists t_drop_columns_with_idx") tk.MustExec("create index idx_1 on t_drop_columns_with_idx(b)") tk.MustExec("create index idx_2 on t_drop_columns_with_idx(b)") tk.MustExec("create index idx_3 on t_drop_columns_with_idx(c)") tk.MustExec("alter table t_drop_columns_with_idx drop column b, drop column c") - query := queryIndexOnTable("test_db", "t_drop_columns_with_idx") + query := queryIndexOnTable("test", "t_drop_columns_with_idx") tk.MustQuery(query).Check(testkit.Rows()) } -func (s *testSerialDBSuite) TestDropLastVisibleColumnOrColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") +func TestDropLastVisibleColumnOrColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create table t_drop_last_column(x int, key((1+1)))") _, err := tk.Exec("alter table t_drop_last_column drop column x") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1113]A table must have at least 1 column") + require.Error(t, err) + require.Equal(t, "[ddl:1113]A table must have at least 1 column", err.Error()) // for visible columns tk.MustExec("create table t_drop_last_columns(x int, y int, key((1+1)))") _, err = tk.Exec("alter table t_drop_last_columns drop column x, drop column y") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:1113]A table must have at least 1 column") + require.Error(t, err) + require.Equal(t, "[ddl:1113]A table must have at least 1 column", err.Error()) tk.MustExec("drop table if exists t_drop_last_column, t_drop_last_columns") } -func (s *testSerialDBSuite1) TestAutoIncrementTableOption(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoIncrementTableOption(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_auto_inc_table_opt;") tk.MustExec("create database test_auto_inc_table_opt;") tk.MustExec("use test_auto_inc_table_opt;") @@ -2698,15 +2694,17 @@ func (s *testSerialDBSuite1) TestAutoIncrementTableOption(c *C) { tk.MustQuery("select * from t;").Check(testkit.Rows("12345678901234567890")) } -func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoIncrementForce(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists auto_inc_force;") tk.MustExec("create database auto_inc_force;") tk.MustExec("use auto_inc_force;") getNextGlobalID := func() uint64 { gidStr := tk.MustQuery("show table t next_row_id").Rows()[0][3] gid, err := strconv.ParseUint(gidStr.(string), 10, 64) - c.Assert(err, IsNil) + require.NoError(t, err) return gid } // Rebase _tidb_row_id. @@ -2717,10 +2715,10 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { // Cannot set next global ID to 0. tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) tk.MustExec("alter table t force auto_increment = 1;") - c.Assert(getNextGlobalID(), Equals, uint64(1)) + require.Equal(t, uint64(1), getNextGlobalID()) // inserting new rows can overwrite the existing data. tk.MustExec("insert into t values (3);") - c.Assert(tk.ExecToErr("insert into t values (3);").Error(), Equals, "[kv:1062]Duplicate entry '2' for key 'PRIMARY'") + require.Equal(t, "[kv:1062]Duplicate entry '2' for key 'PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) // Rebase auto_increment. @@ -2729,11 +2727,11 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { tk.MustExec("insert into t values (1, 1);") tk.MustExec("insert into t values (100000000, 1);") tk.MustExec("delete from t where a = 100000000;") - c.Assert(getNextGlobalID(), Greater, uint64(100000000)) + require.Greater(t, getNextGlobalID(), uint64(100000000)) // Cannot set next global ID to 0. tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") - c.Assert(getNextGlobalID(), Equals, uint64(2)) + require.Equal(t, uint64(2), getNextGlobalID()) tk.MustExec("insert into t(b) values (2);") tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) @@ -2744,11 +2742,11 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { tk.MustExec("set @@allow_auto_random_explicit_insert = true") tk.MustExec("insert into t values (100000000);") tk.MustExec("delete from t where a = 100000000;") - c.Assert(getNextGlobalID(), Greater, uint64(100000000)) + require.Greater(t, getNextGlobalID(), uint64(100000000)) // Cannot set next global ID to 0. tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) tk.MustExec("alter table t force auto_random_base = 2;") - c.Assert(getNextGlobalID(), Equals, uint64(2)) + require.Equal(t, uint64(2), getNextGlobalID()) tk.MustExec("insert into t values ();") tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) @@ -2760,7 +2758,7 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) for _, b := range bases { tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - c.Assert(getNextGlobalID(), Equals, b) + require.Equal(t, b, getNextGlobalID()) } tk.MustExec("insert into t values ();") tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) @@ -2769,7 +2767,7 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { tk.MustExec("create table t (a bigint unsigned primary key auto_increment);") for _, b := range bases { tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - c.Assert(getNextGlobalID(), Equals, b) + require.Equal(t, b, getNextGlobalID()) tk.MustExec("insert into t values ();") tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) tk.MustExec("delete from t;") @@ -2788,8 +2786,10 @@ func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { tk.MustExec("drop table if exists t;") } -func (s *testIntegrationSuite3) TestIssue20490(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20490(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("create table issue20490 (a int);") tk.MustExec("insert into issue20490(a) values(1);") @@ -2801,8 +2801,10 @@ func (s *testIntegrationSuite3) TestIssue20490(c *C) { tk.MustQuery("select b from issue20490 order by a;").Check(testkit.Rows("1", "1", "")) } -func (s *testIntegrationSuite3) TestIssue20741WithEnumField(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20741WithEnumField(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists issue20741") tk.MustExec("create table issue20741(id int primary key, c int)") @@ -2814,8 +2816,10 @@ func (s *testIntegrationSuite3) TestIssue20741WithEnumField(c *C) { tk.MustQuery("select * from issue20741 where cc = 1").Check(testkit.Rows("1 2 a", "2 2 a")) } -func (s *testIntegrationSuite3) TestIssue20741WithSetField(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20741WithSetField(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists issue20741_2") tk.MustExec("create table issue20741_2(id int primary key, c int)") @@ -2826,44 +2830,50 @@ func (s *testIntegrationSuite3) TestIssue20741WithSetField(c *C) { tk.MustQuery("select * from issue20741_2 where cc = 0").Check(testkit.Rows("1 2 ", "2 2 ")) tk.MustQuery("select * from issue20741_2 where cc = 1").Check(testkit.Rows()) _, err := tk.Exec("insert into issue20741_2(id, c) values (3, 3)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[table:1364]Field 'cc' doesn't have a default value") + require.Error(t, err) + require.Equal(t, "[table:1364]Field 'cc' doesn't have a default value", err.Error()) } // TestDefaultValueIsLatin1 for issue #18977 -func (s *testIntegrationSuite3) TestEnumAndSetDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnumAndSetDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (a enum(0x61, 'b') not null default 0x61, b set(0x61, 'b') not null default 0x61) character set latin1") - tbl := testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl.Meta().Columns[0].DefaultValue, Equals, "a") - c.Assert(tbl.Meta().Columns[1].DefaultValue, Equals, "a") + tbl := testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.Equal(t, "a", tbl.Meta().Columns[0].DefaultValue) + require.Equal(t, "a", tbl.Meta().Columns[1].DefaultValue) tk.MustExec("drop table t") tk.MustExec("create table t (a enum(0x61, 'b') not null default 0x61, b set(0x61, 'b') not null default 0x61) character set utf8mb4") - tbl = testGetTableByName(c, s.ctx, "test", "t") - c.Assert(tbl.Meta().Columns[0].DefaultValue, Equals, "a") - c.Assert(tbl.Meta().Columns[1].DefaultValue, Equals, "a") + tbl = testkit.TestGetTableByName(t, tk.Session(), "test", "t") + require.Equal(t, "a", tbl.Meta().Columns[0].DefaultValue) + require.Equal(t, "a", tbl.Meta().Columns[1].DefaultValue) } -func (s *testIntegrationSuite3) TestStrictDoubleTypeCheck(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStrictDoubleTypeCheck(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'") sql := "create table double_type_check(id int, c double(10));" _, err := tk.Exec(sql) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[parser:1149]You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use") + require.Error(t, err) + require.Equal(t, "[parser:1149]You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use", err.Error()) tk.MustExec("set @@tidb_enable_strict_double_type_check = 'OFF'") defer tk.MustExec("set @@tidb_enable_strict_double_type_check = 'ON'") tk.MustExec(sql) } -func (s *testSerialDBSuite) TestDuplicateErrorMessage(c *C) { - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) +func TestDuplicateErrorMessage(t *testing.T) { + defer collate.SetNewCollationEnabledForTest(true) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") type testdata struct { @@ -2886,7 +2896,7 @@ func (s *testSerialDBSuite) TestDuplicateErrorMessage(c *C) { conf.EnableGlobalIndex = globalIndex }) for _, clusteredIndex := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} { - tk.Se.GetSessionVars().EnableClusteredIndex = clusteredIndex + tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex for _, t := range tests { tk.MustExec("drop table if exists t;") fields := make([]string, len(t.types)) @@ -2916,29 +2926,35 @@ func (s *testSerialDBSuite) TestDuplicateErrorMessage(c *C) { } } -func (s *testIntegrationSuite3) TestIssue22028(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22028(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") _, err := tk.Exec("create table t(a double(0, 0));") - c.Assert(err.Error(), Equals, "[types:1439]Display width out of range for column 'a' (max = 255)") + require.Equal(t, "[types:1439]Display width out of range for column 'a' (max = 255)", err.Error()) tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a double);") _, err = tk.Exec("ALTER TABLE t MODIFY COLUMN a DOUBLE(0,0);") - c.Assert(err.Error(), Equals, "[types:1439]Display width out of range for column 'a' (max = 255)") + require.Equal(t, "[types:1439]Display width out of range for column 'a' (max = 255)", err.Error()) } -func (s *testIntegrationSuite3) TestIssue21835(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue21835(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") _, err := tk.Exec("create table t( col decimal(1,2) not null default 0);") - c.Assert(err.Error(), Equals, "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'col').") + require.Equal(t, "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'col').", err.Error()) } -func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTemporaryTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") @@ -2977,7 +2993,7 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { tk.MustQuery("select * from t1") // No error tk.MustExec("drop database tmp_db") _, err := tk.Exec("select * from t1") - c.Assert(err, NotNil) + require.Error(t, err) // In MySQL, drop DB does not really drop the table, it's back! tk.MustExec("create database tmp_db") tk.MustExec("use tmp_db") @@ -2987,7 +3003,7 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { tk.MustExec("create table overlap (id int)") tk.MustExec("create temporary table overlap (a int, b int)") _, err = tk.Exec("insert into overlap values (1)") // column not match - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("insert into overlap values (1, 1)") // Check create local temporary table does not auto commit the transaction. @@ -3004,7 +3020,7 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { // Check create temporary table for if not exists tk.MustExec("create temporary table b_local_temp_table (id int)") _, err = tk.Exec("create temporary table b_local_temp_table (id int)") - c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableExists.Equal(err)) tk.MustExec("create temporary table if not exists b_local_temp_table (id int)") // Stale read see the local temporary table but can't read on it. @@ -3022,8 +3038,10 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { tk.MustExec(updateSafePoint) } -func (s *testIntegrationSuite3) TestAccessLocalTmpTableAfterDropDB(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAccessLocalTmpTableAfterDropDB(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists tmpdb") tk.MustExec("create temporary table tmpdb.tmp(id int)") tk.MustExec("drop database tmpdb") @@ -3105,8 +3123,10 @@ func (s *testIntegrationSuite3) TestAccessLocalTmpTableAfterDropDB(c *C) { executeTests() } -func (s *testSerialDBSuite) TestPlacementOnTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPlacementOnTemporaryTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists test.tplacement1, db2.t1, db2.tplacement3, db2.tplacement5") tk.MustExec("drop database if exists db2") @@ -3164,11 +3184,13 @@ func (s *testSerialDBSuite) TestPlacementOnTemporaryTable(c *C) { )) } -func (s *testIntegrationSuite3) TestAvoidCreateViewOnLocalTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAvoidCreateViewOnLocalTemporaryTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) tk.MustExec("drop table if exists tt0") tk.MustExec("drop table if exists tt1") tk.MustExec("drop table if exists tt2") @@ -3180,55 +3202,55 @@ func (s *testIntegrationSuite3) TestAvoidCreateViewOnLocalTemporaryTable(c *C) { checkCreateView := func() { _, err := tk.Exec("create view v1 as select * from tt1") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v1' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v1' doesn't exist", err.Error()) _, err = tk.Exec("create view v1 as select * from (select * from tt1) as tt") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v1' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v1' doesn't exist", err.Error()) _, err = tk.Exec("create view v2 as select * from tt0 union select * from tt1") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v2") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v2' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v2' doesn't exist", err.Error()) _, err = tk.Exec("create view v3 as select * from tt0, tt1 where tt0.a = tt1.a") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v3") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v3' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v3' doesn't exist", err.Error()) _, err = tk.Exec("create view v4 as select a, (select count(1) from tt1 where tt1.a = tt0.a) as tt1a from tt0") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v4") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v4' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v4' doesn't exist", err.Error()) _, err = tk.Exec("create view v5 as select a, (select count(1) from tt1 where tt1.a = 1) as tt1a from tt0") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v5") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v5' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v5' doesn't exist", err.Error()) _, err = tk.Exec("create view v6 as select * from tt0 where tt0.a=(select max(tt1.b) from tt1)") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v6") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v6' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v6' doesn't exist", err.Error()) _, err = tk.Exec("create view v7 as select * from tt0 where tt0.b=(select max(tt1.b) from tt1 where tt0.a=tt1.a)") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) _, err = tk.Exec("select * from v7") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.v7' doesn't exist") + require.Error(t, err) + require.Equal(t, "[schema:1146]Table 'test.v7' doesn't exist", err.Error()) _, err = tk.Exec("create or replace view v0 as select * from tt1") - c.Assert(core.ErrViewSelectTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrViewSelectTemporaryTable.Equal(err)) } checkCreateView() @@ -3239,8 +3261,18 @@ func (s *testIntegrationSuite3) TestAvoidCreateViewOnLocalTemporaryTable(c *C) { checkCreateView() } -func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropTemporaryTable(t *testing.T) { + config.UpdateGlobal(func(conf *config.Config) { + // Test for table lock. + conf.EnableTableLock = true + conf.Log.SlowThreshold = 10000 + conf.TiKVClient.AsyncCommit.SafeWindow = 0 + conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 + conf.Experimental.AllowsExpressionIndex = true + }) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Check drop temporary table(include meta data and real data. @@ -3248,7 +3280,7 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { tk.MustQuery("select * from b_local_temp_table").Check(testkit.Rows()) tk.MustExec("drop table b_local_temp_table") _, err := tk.Exec("select * from b_local_temp_table") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.b_local_temp_table' doesn't exist") + require.Equal(t, "[schema:1146]Table 'test.b_local_temp_table' doesn't exist", err.Error()) // TODO: test drop real data // Check if we have a normal and local temporary table in the same db with the same name, @@ -3258,11 +3290,11 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { tk.MustExec("create temporary table if not exists b_table_local_and_normal (id int)") tk.MustQuery("select * from b_table_local_and_normal").Check(testkit.Rows()) tk.MustExec("drop table b_table_local_and_normal") - sequenceTable := testGetTableByName(c, tk.Se, "test", "b_table_local_and_normal") - c.Assert(sequenceTable.Meta().TempTableType, Equals, model.TempTableNone) + sequenceTable := testkit.TestGetTableByName(t, tk.Session(), "test", "b_table_local_and_normal") + require.Equal(t, model.TempTableNone, sequenceTable.Meta().TempTableType) tk.MustExec("drop table if exists b_table_local_and_normal") _, err = tk.Exec("select * from b_table_local_and_normal") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.b_table_local_and_normal' doesn't exist") + require.Equal(t, "[schema:1146]Table 'test.b_table_local_and_normal' doesn't exist", err.Error()) // Check dropping local temporary tables should not commit current transaction implicitly. tk.MustExec("drop table if exists check_data_normal_table") @@ -3308,30 +3340,26 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { tk.MustExec("create table a_normal_table_2 (id int)") defer tk.MustExec("drop table if exists a_normal_table_2") _, err = tk.Exec("drop table a_local_temp_table_3, a_local_temp_table_4, a_local_temp_table_5, a_normal_table_2, a_local_temp_table_6") - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.a_local_temp_table_6'") + require.Equal(t, "[schema:1051]Unknown table 'test.a_local_temp_table_6'", err.Error()) tk.MustExec("drop table if exists check_data_normal_table_3") tk.MustExec("create table check_data_normal_table_3 (id int)") defer tk.MustExec("drop table if exists check_data_normal_table_3") tk.MustExec("create temporary table a_local_temp_table_6 (id int)") _, err = tk.Exec("drop table check_data_normal_table_3, check_data_normal_table_7, a_local_temp_table_6") - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.check_data_normal_table_7'") + require.Equal(t, "[schema:1051]Unknown table 'test.check_data_normal_table_7'", err.Error()) // Check filter out data from removed local temp tables tk.MustExec("create temporary table a_local_temp_table_7 (id int)") - ctx := s.ctx - c.Assert(ctx.NewTxn(context.Background()), IsNil) - txn, err := ctx.Txn(true) - c.Assert(err, IsNil) - defer func() { - err := txn.Rollback() - c.Assert(err, IsNil) - }() - sessionVars := tk.Se.GetSessionVars() + ctx := tk.Session() + require.Nil(t, ctx.NewTxn(context.Background())) + _, err = ctx.Txn(true) + require.NoError(t, err) + sessionVars := tk.Session().GetSessionVars() sessVarsTempTable := sessionVars.LocalTemporaryTables localTemporaryTable := sessVarsTempTable.(*infoschema.LocalTemporaryTables) tbl, exist := localTemporaryTable.TableByName(model.NewCIStr("test"), model.NewCIStr("a_local_temp_table_7")) - c.Assert(exist, IsTrue) + require.True(t, exist) tblInfo := tbl.Meta() tablePrefix := tablecodec.EncodeTablePrefix(tblInfo.ID) endTablePrefix := tablecodec.EncodeTablePrefix(tblInfo.ID + 1) @@ -3344,31 +3372,33 @@ func (s *testIntegrationSuite3) TestDropTemporaryTable(c *C) { tk.MustExec("commit") _, err = tk.Exec("select * from a_local_temp_table_7") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.a_local_temp_table_7' doesn't exist") + require.Equal(t, "[schema:1146]Table 'test.a_local_temp_table_7' doesn't exist", err.Error()) memData := sessionVars.TemporaryTableData iter, err := memData.Iter(tablePrefix, endTablePrefix) - c.Assert(err, IsNil) + require.NoError(t, err) for iter.Valid() { key := iter.Key() if !bytes.HasPrefix(key, tablePrefix) { break } value := iter.Value() - c.Assert(len(value), Equals, 0) + require.Equal(t, 0, len(value)) _ = iter.Next() } - c.Assert(iter.Valid(), IsFalse) + require.False(t, iter.Valid()) // Check drop not exists table in transaction. tk.MustExec("begin") tk.MustExec("create temporary table a_local_temp_table_8 (id int)") _, err = tk.Exec("drop table a_local_temp_table_8, a_local_temp_table_9_not_exist") - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.a_local_temp_table_9_not_exist'") + require.Equal(t, "[schema:1051]Unknown table 'test.a_local_temp_table_9_not_exist'", err.Error()) tk.MustQuery("select * from a_local_temp_table_8").Check(testkit.Rows()) } -func (s *testIntegrationSuite3) TestDropWithGlobalTemporaryTableKeyWord(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropWithGlobalTemporaryTableKeyWord(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") clearSQL := "drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2" tk.MustExec(clearSQL) @@ -3385,51 +3415,51 @@ func (s *testIntegrationSuite3) TestDropWithGlobalTemporaryTableKeyWord(c *C) { // testing for drop table which is not global temporary err := tk.ExecToErr("drop global temporary table tb") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table test.tb") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table ltemp1") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table test.ltemp1") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table ltemp1, temp") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table temp, ltemp1") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table xxx, ltemp1") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table xxx") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) // testing for drop table if exists which is not global temporary err = tk.ExecToErr("drop global temporary table if exists tb") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table if exists ltemp1") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) tk.MustExec("drop global temporary table if exists xxx") tk.MustQuery("show warnings").Check(testkit.Rows("Note 1051 Unknown table 'test.xxx'")) err = tk.ExecToErr("drop global temporary table if exists xxx,tb") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) err = tk.ExecToErr("drop global temporary table if exists test.tb") - c.Assert(core.ErrDropTableOnTemporaryTable.Equal(err), IsTrue) + require.True(t, core.ErrDropTableOnTemporaryTable.Equal(err)) // testing for drop global temporary table successfully tk.MustExec("drop global temporary table temp") err = tk.ExecToErr("select * from temp") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("drop global temporary table test.temp1") err = tk.ExecToErr("select * from temp2") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("create global temporary table temp (id int) on commit delete rows") tk.MustExec("create global temporary table temp1 (id int) on commit delete rows") tk.MustExec("drop global temporary table temp, temp1") err = tk.ExecToErr("select * from temp") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) err = tk.ExecToErr("select * from temp1") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("create global temporary table temp (id int) on commit delete rows") tk.MustExec("create global temporary table temp1 (id int) on commit delete rows") @@ -3437,11 +3467,13 @@ func (s *testIntegrationSuite3) TestDropWithGlobalTemporaryTableKeyWord(c *C) { tk.MustExec("drop global temporary table if exists temp") tk.MustQuery("show warnings").Check(testkit.Rows()) err = tk.ExecToErr("select * from temp") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) } -func (s *testIntegrationSuite3) TestDropWithLocalTemporaryTableKeyWord(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropWithLocalTemporaryTableKeyWord(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") clearSQL := "drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2, testt.ltemp3" tk.MustExec(clearSQL) @@ -3463,21 +3495,21 @@ func (s *testIntegrationSuite3) TestDropWithLocalTemporaryTableKeyWord(c *C) { // testing for drop table which is not local temporary err := tk.ExecToErr("drop temporary table tb") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table test.tb") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table temp1") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table test.temp1") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table ltemp1, tb") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table temp, ltemp1") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table xxx, ltemp1") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) err = tk.ExecToErr("drop temporary table xxx") - c.Assert(infoschema.ErrTableDropExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableDropExists.Equal(err)) // testing for drop table if exists which is not local temporary tk.MustExec("drop temporary table if exists xxx") @@ -3498,11 +3530,11 @@ func (s *testIntegrationSuite3) TestDropWithLocalTemporaryTableKeyWord(c *C) { // testing for drop temporary table successfully tk.MustExec("drop temporary table ltemp1") err = tk.ExecToErr("select * from ltemp1") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("drop temporary table test.ltemp2") err = tk.ExecToErr("select * from ltemp2") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("drop temporary table tb2") tk.MustQuery("select * from tb2").Check(testkit.Rows("1")) @@ -3512,18 +3544,20 @@ func (s *testIntegrationSuite3) TestDropWithLocalTemporaryTableKeyWord(c *C) { tk.MustExec("drop temporary table testt.ltemp3, ltemp1") err = tk.ExecToErr("select * from testt.ltemp3") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) err = tk.ExecToErr("select * from ltemp1") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) tk.MustExec("drop temporary table if exists ltemp2") tk.MustQuery("show warnings").Check(testkit.Rows()) err = tk.ExecToErr("select * from ltemp2") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) } -func (s *testIntegrationSuite3) TestTruncateLocalTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTruncateLocalTemporaryTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, tn") @@ -3587,28 +3621,28 @@ func (s *testIntegrationSuite3) TestTruncateLocalTemporaryTable(c *C) { tk.MustExec("create temporary table t1 (id int primary key auto_increment)") // truncate temporary table will clear session data - localTemporaryTables := tk.Se.GetSessionVars().LocalTemporaryTables.(*infoschema.LocalTemporaryTables) + localTemporaryTables := tk.Session().GetSessionVars().LocalTemporaryTables.(*infoschema.LocalTemporaryTables) tb1, exist := localTemporaryTables.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) tbl1Info := tb1.Meta() tablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID) endTablePrefix := tablecodec.EncodeTablePrefix(tbl1Info.ID + 1) - c.Assert(exist, IsTrue) + require.True(t, exist) tk.MustExec("insert into t1 values(1), (2), (3)") tk.MustExec("begin") tk.MustExec("insert into t1 values(5), (6), (7)") tk.MustExec("truncate table t1") - iter, err := tk.Se.GetSessionVars().TemporaryTableData.Iter(tablePrefix, endTablePrefix) - c.Assert(err, IsNil) + iter, err := tk.Session().GetSessionVars().TemporaryTableData.Iter(tablePrefix, endTablePrefix) + require.NoError(t, err) for iter.Valid() { key := iter.Key() if !bytes.HasPrefix(key, tablePrefix) { break } value := iter.Value() - c.Assert(len(value), Equals, 0) + require.Equal(t, 0, len(value)) _ = iter.Next() } - c.Assert(iter.Valid(), IsFalse) + require.False(t, iter.Valid()) // truncate after drop database should be successful tk.MustExec("create temporary table testt.t3 (id int)") @@ -3618,9 +3652,11 @@ func (s *testIntegrationSuite3) TestTruncateLocalTemporaryTable(c *C) { tk.MustQuery("select * from testt.t3").Check(testkit.Rows()) } -func (s *testIntegrationSuite3) TestIssue29282(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk1 := testkit.NewTestKit(c, s.store) +func TestIssue29282(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk1 := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists issue29828_t") tk.MustExec("create table issue29828_t (id int)") @@ -3642,6 +3678,7 @@ func (s *testIntegrationSuite3) TestIssue29282(c *C) { // This query should block. tk1.MustQuery("select * from issue29828_t where id = 1 for update;").Check(testkit.Rows("1")) ch <- struct{}{} + tk1.MustExec("rollback") }() select { @@ -3650,28 +3687,32 @@ func (s *testIntegrationSuite3) TestIssue29282(c *C) { tk.MustExec("rollback") case <-ch: // Unexpected, test fail. - c.Fail() + t.Fail() } } // See https://github.com/pingcap/tidb/issues/29327 -func (s *testIntegrationSuite3) TestEnumDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnumDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") - tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci;") + tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;") tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + - " `a` enum('','a','b') COLLATE utf8mb4_0900_ai_ci NOT NULL DEFAULT 'b'\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci")) + " `a` enum('','a','b') COLLATE utf8mb4_general_ci NOT NULL DEFAULT 'b'\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci")) tk.MustExec("drop table if exists t1;") - tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b ' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci;") + tk.MustExec("CREATE TABLE `t1` ( `a` enum('','a','b') NOT NULL DEFAULT 'b ' ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci;") tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + - " `a` enum('','a','b') COLLATE utf8mb4_0900_ai_ci NOT NULL DEFAULT 'b'\n" + - ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci")) + " `a` enum('','a','b') COLLATE utf8mb4_general_ci NOT NULL DEFAULT 'b'\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci")) } -func (s *testIntegrationSuite3) TestIssue29326(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29326(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -3689,33 +3730,33 @@ func (s *testIntegrationSuite3) TestIssue29326(c *C) { tk.MustExec("create view v1 as select 1,1") rs, err := tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + require.Equal(t, "1", rs.Fields()[0].Column.Name.O) + require.Equal(t, "Name_exp_1", rs.Fields()[1].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create view v1 as select 1, 2, 1, 2, 1, 2, 1, 2") rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 2 1 2 1 2 1 2")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "2") - c.Assert(rs.Fields()[2].Column.Name.O, Equals, "Name_exp_1") - c.Assert(rs.Fields()[3].Column.Name.O, Equals, "Name_exp_2") - c.Assert(rs.Fields()[4].Column.Name.O, Equals, "Name_exp_1_1") - c.Assert(rs.Fields()[5].Column.Name.O, Equals, "Name_exp_1_2") - c.Assert(rs.Fields()[6].Column.Name.O, Equals, "Name_exp_2_1") - c.Assert(rs.Fields()[7].Column.Name.O, Equals, "Name_exp_2_2") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 2 1 2 1 2 1 2")) + require.Equal(t, "1", rs.Fields()[0].Column.Name.O) + require.Equal(t, "2", rs.Fields()[1].Column.Name.O) + require.Equal(t, "Name_exp_1", rs.Fields()[2].Column.Name.O) + require.Equal(t, "Name_exp_2", rs.Fields()[3].Column.Name.O) + require.Equal(t, "Name_exp_1_1", rs.Fields()[4].Column.Name.O) + require.Equal(t, "Name_exp_1_2", rs.Fields()[5].Column.Name.O) + require.Equal(t, "Name_exp_2_1", rs.Fields()[6].Column.Name.O) + require.Equal(t, "Name_exp_2_2", rs.Fields()[7].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create view v1 as select 't', 't', 1 as t") rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("t t 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_t") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1_t") - c.Assert(rs.Fields()[2].Column.Name.O, Equals, "t") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("t t 1")) + require.Equal(t, "Name_exp_t", rs.Fields()[0].Column.Name.O) + require.Equal(t, "Name_exp_1_t", rs.Fields()[1].Column.Name.O) + require.Equal(t, "t", rs.Fields()[2].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1, 1 union all select 1, 1") @@ -3724,10 +3765,10 @@ func (s *testIntegrationSuite3) TestIssue29326(c *C) { "AS SELECT 1 AS `1`,1 AS `Name_exp_1` UNION ALL SELECT 1 AS `1`,1 AS `1` " + "utf8mb4 utf8mb4_bin")) rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1", "1 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "1") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_1") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1", "1 1")) + require.Equal(t, "1", rs.Fields()[0].Column.Name.O) + require.Equal(t, "Name_exp_1", rs.Fields()[1].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 'id', id from t1") @@ -3736,10 +3777,10 @@ func (s *testIntegrationSuite3) TestIssue29326(c *C) { "AS SELECT _UTF8MB4'id' AS `Name_exp_id`,`id` AS `id` FROM `test`.`t1` " + "utf8mb4 utf8mb4_bin")) rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("id 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_id") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "id") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("id 1")) + require.Equal(t, "Name_exp_id", rs.Fields()[0].Column.Name.O) + require.Equal(t, "id", rs.Fields()[1].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1, (select id from t1 where t1.id=t2.id) as '1' from t2") @@ -3748,10 +3789,10 @@ func (s *testIntegrationSuite3) TestIssue29326(c *C) { "AS SELECT 1 AS `Name_exp_1`,(SELECT `id` AS `id` FROM `test`.`t1` WHERE `t1`.`id`=`t2`.`id`) AS `1` FROM `test`.`t2` " + "utf8mb4 utf8mb4_bin")) rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "Name_exp_1") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "1") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + require.Equal(t, "Name_exp_1", rs.Fields()[0].Column.Name.O) + require.Equal(t, "1", rs.Fields()[1].Column.Name.O) tk.MustExec("drop view if exists v1") tk.MustExec("create definer=`root`@`127.0.0.1` view v1 as select 1 as 'abs(t1.id)', abs(t1.id) from t1") @@ -3760,24 +3801,72 @@ func (s *testIntegrationSuite3) TestIssue29326(c *C) { "AS SELECT 1 AS `abs(t1.id)`,ABS(`t1`.`id`) AS `Name_exp_abs(t1.id)` FROM `test`.`t1` " + "utf8mb4 utf8mb4_bin")) rs, err = tk.Exec("select * from v1") - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 1")) - c.Assert(rs.Fields()[0].Column.Name.O, Equals, "abs(t1.id)") - c.Assert(rs.Fields()[1].Column.Name.O, Equals, "Name_exp_abs(t1.id)") + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 1")) + require.Equal(t, "abs(t1.id)", rs.Fields()[0].Column.Name.O) + require.Equal(t, "Name_exp_abs(t1.id)", rs.Fields()[1].Column.Name.O) tk.MustExec("drop view if exists v1") err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select 1 as t,1 as t") - c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrColumnExists.Equal(err)) tk.MustExec("drop view if exists v1") err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select 1 as id, id from t1") - c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrColumnExists.Equal(err)) tk.MustExec("drop view if exists v1") err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select * from t1 left join t2 on t1.id=t2.id") - c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrColumnExists.Equal(err)) tk.MustExec("drop view if exists v1") err = tk.ExecToErr("create definer=`root`@`127.0.0.1` view v1 as select t1.id, t2.id from t1,t2 where t1.id=t2.id") - c.Assert(infoschema.ErrColumnExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrColumnExists.Equal(err)) +} + +func TestInvalidPartitionNameWhenCreateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database invalidPartitionNames") + defer tk.MustExec("drop database invalidPartitionNames") + tk.MustExec("USE invalidPartitionNames") + + _, err := tk.Exec("create table t(a int) partition by range (a) (partition p0 values less than (0), partition `p1 ` values less than (3))") + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err) + + _, err = tk.Exec("create table t(a int) partition by range (a) (partition `` values less than (0), partition `p1` values less than (3))") + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err) + + tk.MustExec("create table t(a int) partition by range (a) (partition `p0` values less than (0), partition `p1` values less than (3))") + _, err = tk.Exec("alter table t add partition (partition `p2 ` values less than (5))") + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, ddl.ErrWrongPartitionName), "err %v", err) +} + +func TestDDLLastInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test;`) + tk.MustQuery("select json_extract(@@tidb_last_ddl_info, '$.query'), json_extract(@@tidb_last_ddl_info, '$.seq_num')").Check(testkit.Rows("\"\" 0")) + tk.MustExec("create table t(a int)") + firstSequence := 0 + res := tk.MustQuery("select json_extract(@@tidb_last_ddl_info, '$.query'), json_extract(@@tidb_last_ddl_info, '$.seq_num')") + require.Len(t, res.Rows(), 1) + require.Equal(t, "\"create table t(a int)\"", res.Rows()[0][0]) + var err error + firstSequence, err = strconv.Atoi(fmt.Sprintf("%v", res.Rows()[0][1])) + require.NoError(t, err) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use test;`) + tk.MustExec("create table t2(a int)") + tk.MustQuery("select json_extract(@@tidb_last_ddl_info, '$.query'), json_extract(@@tidb_last_ddl_info, '$.seq_num')").Check(testkit.Rows(fmt.Sprintf("\"create table t2(a int)\" %d", firstSequence+1))) + + tk.MustExec("drop table t, t2") + tk.MustQuery("select json_extract(@@tidb_last_ddl_info, '$.query'), json_extract(@@tidb_last_ddl_info, '$.seq_num')").Check(testkit.Rows(fmt.Sprintf("\"drop table t, t2\" %d", firstSequence+3))) } diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 393a720411c4f..b5121c5853aef 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -48,7 +48,6 @@ import ( ntestkit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" @@ -56,6 +55,25 @@ import ( "go.uber.org/zap" ) +var _ = Suite(&testIntegrationSuite1{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite2{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) + +type testIntegrationSuite1 struct{ *testIntegrationSuite } +type testIntegrationSuite2 struct{ *testIntegrationSuite } + +func (s *testIntegrationSuite2) TearDownTest(c *C) { + tearDownIntegrationSuiteTest(s.testIntegrationSuite, c) +} + +type testIntegrationSuite3 struct{ *testIntegrationSuite } +type testIntegrationSuite4 struct{ *testIntegrationSuite } +type testIntegrationSuite5 struct{ *testIntegrationSuite } +type testIntegrationSuite6 struct{ *testIntegrationSuite } + func (s *testIntegrationSuite3) TestCreateTableWithPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") @@ -367,8 +385,6 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { } func (s *testSerialDBSuite1) TestCreateTableWithRangeColumnPartition(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists log_message_1;") @@ -901,6 +917,14 @@ func (s *testIntegrationSuite1) TestCreateTableWithListColumnsPartition(c *C) { "create table t (a bigint, b int) partition by list columns (a,b) (partition p0 values in ((1,1),(2,2)), partition p1 values in ((+1,1)));", ddl.ErrMultipleDefConstInListPart, }, + { + "create table t1 (a int, b int) partition by list columns(a,a) ( partition p values in ((1,1)));", + ddl.ErrSameNamePartitionField, + }, + { + "create table t1 (a int, b int) partition by list columns(a,b,b) ( partition p values in ((1,1,1)));", + ddl.ErrSameNamePartitionField, + }, { `create table t1 (id int key, name varchar(10), unique index idx(name)) partition by list columns (id) ( partition p0 values in (3,5,6,9,17), @@ -3380,8 +3404,6 @@ func (s *testSerialDBSuite1) TestPartitionListWithTimeType(c *C) { } func (s *testSerialDBSuite1) TestPartitionListWithNewCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") @@ -3553,6 +3575,13 @@ func TestRenameTables(t *testing.T) { ddl.ExportTestRenameTables(t) } +func TestCreateTables(t *testing.T) { + _, clean := ntestkit.CreateMockStore(t) + defer clean() + + ddl.ExportTestRenameTables(t) +} + func (s *testIntegrationSuite1) TestDuplicatePartitionNames(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/db_test.go b/ddl/db_test.go index 5968e7b920144..86fcf4b25afe4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -15,9 +15,9 @@ package ddl_test import ( + "bytes" "context" "fmt" - "io" "math" "math/rand" "sort" @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" testddlutil "github.com/pingcap/tidb/ddl/testutil" + ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" @@ -58,7 +59,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/domainutil" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" @@ -1451,22 +1451,7 @@ LOOP: c.Assert(ctx.NewTxn(context.Background()), IsNil) - it, err := nidx.SeekFirst(txn) - c.Assert(err, IsNil) - defer it.Close() - - for { - _, h, err := it.Next() - if terror.ErrorEqual(err, io.EOF) { - break - } - - c.Assert(err, IsNil) - _, ok := handles.Get(h) - c.Assert(ok, IsTrue, Commentf("handle: %v", h.String())) - handles.Delete(h) - } - c.Assert(handles.Len(), Equals, 0) + tk.MustExec("admin check table test_add_index") tk.MustExec("drop table test_add_index") } @@ -2861,8 +2846,8 @@ func (s *testSerialDBSuite) TestCreateTable(c *C) { tk.MustGetErrCode("CREATE TABLE `t` (`a` int) DEFAULT CHARSET=abcdefg", errno.ErrUnknownCharacterSet) - tk.MustExec("CREATE TABLE `collateTest` (`a` int, `b` varchar(10)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_slovak_ci") - expects := "collateTest CREATE TABLE `collateTest` (\n `a` int(11) DEFAULT NULL,\n `b` varchar(10) COLLATE utf8_slovak_ci DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_slovak_ci" + tk.MustExec("CREATE TABLE `collateTest` (`a` int, `b` varchar(10)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci") + expects := "collateTest CREATE TABLE `collateTest` (\n `a` int(11) DEFAULT NULL,\n `b` varchar(10) COLLATE utf8_general_ci DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci" tk.MustQuery("show create table collateTest").Check(testkit.Rows(expects)) tk.MustGetErrCode("CREATE TABLE `collateTest2` (`a` int) CHARSET utf8 COLLATE utf8mb4_unicode_ci", errno.ErrCollationCharsetMismatch) @@ -2882,8 +2867,6 @@ func (s *testSerialDBSuite) TestCreateTable(c *C) { tk.MustExec("use test") failSQL := "create table t_enum (a enum('e','e'));" tk.MustGetErrCode(failSQL, errno.ErrDuplicatedValueInType) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk = testkit.NewTestKit(c, s.store) tk.MustExec("use test") failSQL = "create table t_enum (a enum('e','E')) charset=utf8 collate=utf8_general_ci;" @@ -3188,8 +3171,8 @@ func (s *testDBSuite2) TestCreateTableWithSetCol(c *C) { tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault) failedSQL = "create table t_set (a set('1', '4', '10') default '1,4,11');" tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault) - failedSQL = "create table t_set (a set('1', '4', '10') default '1 ,4');" - tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault) + // Success when the new collation is enabled. + tk.MustExec("create table t_set (a set('1', '4', '10') default '1 ,4');") // The type of default value is int. failedSQL = "create table t_set (a set('1', '4', '10') default 0);" tk.MustGetErrCode(failedSQL, errno.ErrInvalidDefault) @@ -3198,6 +3181,7 @@ func (s *testDBSuite2) TestCreateTableWithSetCol(c *C) { // The type of default value is int. // It's for successful cases + tk.MustExec("drop table if exists t_set") tk.MustExec("create table t_set (a set('1', '4', '10', '21') default 1);") tk.MustQuery("show create table t_set").Check(testkit.Rows("t_set CREATE TABLE `t_set` (\n" + " `a` set('1','4','10','21') DEFAULT '1'\n" + @@ -3548,6 +3532,7 @@ func (s *testDBSuite5) TestAlterTableRenameTable(c *C) { func (s *testDBSuite) testRenameTable(c *C, sql string, isAlterTable bool) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.MustGetErrCode("rename table tb1 to tb2;", errno.ErrNoSuchTable) // for different databases tk.MustExec("create table t (c1 int, c2 int)") tk.MustExec("insert t values (1, 1), (2, 2)") @@ -3587,19 +3572,19 @@ func (s *testDBSuite) testRenameTable(c *C, sql string, isAlterTable bool) { if isAlterTable { tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } else { - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } failSQL = fmt.Sprintf(sql, "test.test_not_exist", "test.test_not_exist") if isAlterTable { tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } else { - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } failSQL = fmt.Sprintf(sql, "test.t_not_exist", "test_not_exist.t") if isAlterTable { tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } else { - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } failSQL = fmt.Sprintf(sql, "test1.t2", "test_not_exist.t") tk.MustGetErrCode(failSQL, errno.ErrErrorOnRename) @@ -3624,7 +3609,7 @@ func (s *testDBSuite) testRenameTable(c *C, sql string, isAlterTable bool) { if isAlterTable { tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } else { - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) } // for the same table name @@ -3743,13 +3728,13 @@ func (s *testDBSuite1) TestRenameMultiTables(c *C) { // for failure case failSQL := "rename table test_not_exist.t to test_not_exist.t, test_not_exist.t to test_not_exist.t" - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) failSQL = "rename table test.test_not_exist to test.test_not_exist, test.test_not_exist to test.test_not_exist" - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) failSQL = "rename table test.t_not_exist to test_not_exist.t, test.t_not_exist to test_not_exist.t" - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) failSQL = "rename table test1.t2 to test_not_exist.t, test1.t2 to test_not_exist.t" - tk.MustGetErrCode(failSQL, errno.ErrFileNotFound) + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) tk.MustExec("drop database test1") tk.MustExec("drop database test") @@ -7674,7 +7659,7 @@ func (s *testDBSuite8) TestCreateTextAdjustLen(c *C) { tk.MustExec("drop table if exists t") } -func (s *testDBSuite2) TestCreateTables(c *C) { +func (s *testDBSuite2) TestBatchCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists tables_1") @@ -7710,6 +7695,33 @@ func (s *testDBSuite2) TestCreateTables(c *C) { infos[1].Name = model.NewCIStr("tables_1") err = d.BatchCreateTableWithInfo(tk.Se, model.NewCIStr("test"), infos, ddl.OnExistError) c.Check(terror.ErrorEqual(err, infoschema.ErrTableExists), IsTrue) + + newinfo := &model.TableInfo{ + Name: model.NewCIStr("tables_4"), + } + { + colNum := 2 + cols := make([]*model.ColumnInfo, colNum) + viewCols := make([]model.CIStr, colNum) + var stmtBuffer bytes.Buffer + stmtBuffer.WriteString("SELECT ") + for i := range cols { + col := &model.ColumnInfo{ + Name: model.NewCIStr(fmt.Sprintf("c%d", i+1)), + Offset: i, + State: model.StatePublic, + } + cols[i] = col + viewCols[i] = col.Name + stmtBuffer.WriteString(cols[i].Name.L + ",") + } + stmtBuffer.WriteString("1 FROM t") + newinfo.Columns = cols + newinfo.View = &model.ViewInfo{Cols: viewCols, Security: model.SecurityDefiner, Algorithm: model.AlgorithmMerge, SelectStmt: stmtBuffer.String(), CheckOption: model.CheckOptionCascaded, Definer: &auth.UserIdentity{CurrentUser: true}} + } + + err = d.BatchCreateTableWithInfo(tk.Se, model.NewCIStr("test"), []*model.TableInfo{newinfo}, ddl.OnExistError) + c.Check(err, IsNil) } func (s *testSerialDBSuite) TestAddGeneratedColumnAndInsert(c *C) { @@ -7760,3 +7772,43 @@ func (s *testSerialDBSuite) TestAddGeneratedColumnAndInsert(c *C) { tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("4 5", "10 11")) c.Assert(checkErr, IsNil) } + +func (s *testDBSuite1) TestGetTimeZone(c *C) { + tk := testkit.NewTestKit(c, s.store) + + testCases := []struct { + tzSQL string + tzStr string + tzName string + offset int + err string + }{ + {"set time_zone = '+00:00'", "", "UTC", 0, ""}, + {"set time_zone = '-00:00'", "", "UTC", 0, ""}, + {"set time_zone = 'UTC'", "UTC", "UTC", 0, ""}, + {"set time_zone = '+05:00'", "", "UTC", 18000, ""}, + {"set time_zone = '-08:00'", "", "UTC", -28800, ""}, + {"set time_zone = '+08:00'", "", "UTC", 28800, ""}, + {"set time_zone = 'Asia/Shanghai'", "Asia/Shanghai", "Asia/Shanghai", 0, ""}, + {"set time_zone = 'SYSTEM'", "Asia/Shanghai", "Asia/Shanghai", 0, ""}, + {"set time_zone = DEFAULT", "Asia/Shanghai", "Asia/Shanghai", 0, ""}, + {"set time_zone = 'GMT'", "GMT", "GMT", 0, ""}, + {"set time_zone = 'GMT+1'", "GMT", "GMT", 0, "[variable:1298]Unknown or incorrect time zone: 'GMT+1'"}, + {"set time_zone = 'Etc/GMT+12'", "Etc/GMT+12", "Etc/GMT+12", 0, ""}, + {"set time_zone = 'Etc/GMT-12'", "Etc/GMT-12", "Etc/GMT-12", 0, ""}, + {"set time_zone = 'EST'", "EST", "EST", 0, ""}, + {"set time_zone = 'Australia/Lord_Howe'", "Australia/Lord_Howe", "Australia/Lord_Howe", 0, ""}, + } + for _, tc := range testCases { + err := tk.ExecToErr(tc.tzSQL) + if err != nil { + c.Assert(err.Error(), Equals, tc.err) + } else { + c.Assert(tc.err, Equals, "") + } + c.Assert(tk.Se.GetSessionVars().TimeZone.String(), Equals, tc.tzStr, Commentf("sql: %s", tc.tzSQL)) + tz, offset := ddlutil.GetTimeZone(tk.Se) + c.Assert(tc.tzName, Equals, tz, Commentf("sql: %s, offset: %d", tc.tzSQL, offset)) + c.Assert(tc.offset, Equals, offset, Commentf("sql: %s", tc.tzSQL)) + } +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 3d03d6869adf5..b66e9fd0c4b7c 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -46,9 +46,10 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/table" goutil "github.com/pingcap/tidb/util" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" atomicutil "go.uber.org/atomic" "go.uber.org/zap" ) @@ -185,7 +186,7 @@ type ddl struct { m sync.RWMutex ctx context.Context cancel context.CancelFunc - wg sync.WaitGroup // It's only used to deal with data race in restart_test. + wg tidbutil.WaitGroupWrapper // It's only used to deal with data race in restart_test. limitJobCh chan *limitJobTask *ddlCtx @@ -216,6 +217,11 @@ type ddlCtx struct { hook Callback interceptor Interceptor } + + ddlSeqNumMu struct { + sync.Mutex + seqNum uint64 + } } func (dc *ddlCtx) isOwner() bool { @@ -328,11 +334,11 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ddlCtx.mu.hook = opt.Hook ddlCtx.mu.interceptor = &BaseInterceptor{} d := &ddl{ - ctx: ctx, ddlCtx: ddlCtx, limitJobCh: make(chan *limitJobTask, batchAddingJobs), enableTiFlashPoll: atomicutil.NewBool(true), } + d.ctx, d.cancel = context.WithCancel(ctx) return d } @@ -363,7 +369,6 @@ func (d *ddl) newDeleteRangeManager(mock bool) delRangeManager { // Start implements DDL.Start interface. func (d *ddl) Start(ctxPool *pools.ResourcePool) error { logutil.BgLogger().Info("[ddl] start DDL", zap.String("ID", d.uuid), zap.Bool("runWorker", RunWorker)) - d.ctx, d.cancel = context.WithCancel(d.ctx) d.wg.Add(1) go d.limitDDLJobs() @@ -379,8 +384,8 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.workers = make(map[workerType]*worker, 2) d.sessPool = newSessionPool(ctxPool) d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) - d.workers[generalWorker] = newWorker(d.ctx, generalWorker, d.sessPool, d.delRangeMgr) - d.workers[addIdxWorker] = newWorker(d.ctx, addIdxWorker, d.sessPool, d.delRangeMgr) + d.workers[generalWorker] = newWorker(d.ctx, generalWorker, d.sessPool, d.delRangeMgr, d.ddlCtx) + d.workers[addIdxWorker] = newWorker(d.ctx, addIdxWorker, d.sessPool, d.delRangeMgr, d.ddlCtx) for _, worker := range d.workers { worker.wg.Add(1) w := worker @@ -393,6 +398,15 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { asyncNotify(worker.ddlJobCh) } + err = kv.RunInNewTxn(d.ctx, d.store, true, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + d.ddlSeqNumMu.seqNum, err = t.GetHistoryDDLCount() + return err + }) + if err != nil { + return err + } + go d.schemaSyncer.StartCleanWork() if config.TableLockEnabled() { d.wg.Add(1) @@ -406,7 +420,7 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { metrics.DDLCounter.WithLabelValues(metrics.CreateDDLInstance).Inc() // Start some background routine to manage TiFlash replica. - go d.PollTiFlashRoutine() + d.wg.Run(d.PollTiFlashRoutine) return nil } @@ -570,6 +584,14 @@ func updateTickerInterval(ticker *time.Ticker, lease time.Duration, job *model.J return time.NewTicker(chooseLeaseTime(lease, interval)) } +func recordLastDDLInfo(ctx sessionctx.Context, job *model.Job) { + if job == nil { + return + } + ctx.GetSessionVars().LastDDLInfo.Query = job.Query + ctx.GetSessionVars().LastDDLInfo.SeqNum = job.SeqNum +} + // doDDLJob will return // - nil: found in history DDL job and no job error // - context.Cancel: job has been sent to worker, but not found in history DDL job before cancel @@ -605,6 +627,7 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { ticker.Stop() metrics.JobsGauge.WithLabelValues(job.Type.String()).Dec() metrics.HandleJobHistogram.WithLabelValues(job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + recordLastDDLInfo(ctx, historyJob) }() i := 0 for { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 34219b8f9abad..4407b92deef1c 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/label" + ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -204,6 +205,61 @@ func (d *ddl) ModifySchemaDefaultPlacement(ctx sessionctx.Context, stmt *ast.Alt return errors.Trace(err) } +func (d *ddl) ModifySchemaSetTiFlashReplica(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt, tiflashReplica *ast.TiFlashReplicaSpec) error { + dbName := model.NewCIStr(stmt.Name) + is := d.GetInfoSchemaWithInterceptor(ctx) + dbInfo, ok := is.SchemaByName(dbName) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O) + } + total := len(dbInfo.Tables) + succ := 0 + skip := 0 + fail := 0 + oneFail := int64(0) + + if total == 0 { + return infoschema.ErrEmptyDatabase.GenWithStack("Empty database '%v'", dbName.O) + } + err := checkTiFlashReplicaCount(ctx, tiflashReplica.Count) + if err != nil { + return errors.Trace(err) + } + + for _, tbl := range dbInfo.Tables { + tbReplicaInfo := tbl.TiFlashReplica + if !shouldModifyTiFlashReplica(tbReplicaInfo, tiflashReplica) { + logutil.BgLogger().Info("skip processing schema table", zap.Int64("tableID", tbl.ID), zap.Int64("schemaID", dbInfo.ID), zap.String("tableName", tbl.Name.String()), zap.String("schemaName", dbInfo.Name.String())) + skip += 1 + continue + } + job := &model.Job{ + SchemaID: dbInfo.ID, + SchemaName: dbInfo.Name.L, + TableID: tbl.ID, + Type: model.ActionSetTiFlashReplica, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{*tiflashReplica}, + } + err := d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + if err != nil { + oneFail = tbl.ID + fail += 1 + logutil.BgLogger().Info("processing schema table error", zap.Int64("tableID", tbl.ID), zap.Int64("schemaID", dbInfo.ID), zap.String("tableName", tbl.Name.String()), zap.String("schemaName", dbInfo.Name.String()), zap.Error(err)) + } else { + succ += 1 + } + } + failStmt := "" + if fail > 0 { + failStmt = fmt.Sprintf("(including table %v)", oneFail) + } + msg := fmt.Sprintf("In total %v tables: %v succeed, %v failed%v, %v skipped", total, succ, fail, failStmt, skip) + ctx.GetSessionVars().StmtCtx.SetMessage(msg) + return nil +} + func (d *ddl) AlterTablePlacement(ctx sessionctx.Context, ident ast.Ident, placementPolicyRef *model.PolicyRefInfo) (err error) { is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) @@ -262,14 +318,36 @@ func checkAndNormalizePlacementPolicy(ctx sessionctx.Context, placementPolicyRef return placementPolicyRef, nil } +func checkMultiSchemaSpecs(sctx sessionctx.Context, specs []*ast.DatabaseOption) error { + hasSetTiFlashReplica := false + if len(specs) == 1 { + return nil + } + for _, spec := range specs { + if spec.Tp == ast.DatabaseSetTiFlashReplica { + if hasSetTiFlashReplica { + return errRunMultiSchemaChanges + } + hasSetTiFlashReplica = true + } + } + return nil +} + func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (err error) { // Resolve target charset and collation from options. var ( - toCharset, toCollate string - isAlterCharsetAndCollate, isAlterPlacement bool - placementPolicyRef *model.PolicyRefInfo + toCharset, toCollate string + isAlterCharsetAndCollate, isAlterPlacement, isTiFlashReplica bool + placementPolicyRef *model.PolicyRefInfo + tiflashReplica *ast.TiFlashReplicaSpec ) + err = checkMultiSchemaSpecs(ctx, stmt.Options) + if err != nil { + return err + } + for _, val := range stmt.Options { switch val.Tp { case ast.DatabaseOptionCharset: @@ -294,6 +372,9 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) ( case ast.DatabaseOptionPlacementPolicy: placementPolicyRef = &model.PolicyRefInfo{Name: model.NewCIStr(val.Value)} isAlterPlacement = true + case ast.DatabaseSetTiFlashReplica: + tiflashReplica = val.TiFlashReplica + isTiFlashReplica = true } } @@ -307,7 +388,11 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) ( return err } } - + if isTiFlashReplica { + if err = d.ModifySchemaSetTiFlashReplica(ctx, stmt, tiflashReplica); err != nil { + return err + } + } return nil } @@ -606,7 +691,7 @@ func checkColumnDefaultValue(ctx sessionctx.Context, col *table.Column, value in if value != nil && ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() && ctx.GetSessionVars().SQLMode.HasStrictMode() && types.IsTypeTime(col.Tp) { if vv, ok := value.(string); ok { - timeValue, err := expression.GetTimeValue(ctx, vv, col.Tp, int8(col.Decimal)) + timeValue, err := expression.GetTimeValue(ctx, vv, col.Tp, col.Decimal) if err != nil { return hasDefaultValue, value, errors.Trace(err) } @@ -631,7 +716,7 @@ func convertTimestampDefaultValToUTC(ctx sessionctx.Context, defaultVal interfac } if vv, ok := defaultVal.(string); ok { if vv != types.ZeroDatetimeStr && !strings.EqualFold(vv, ast.CurrentTimestamp) { - t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx, vv, col.Tp, int8(col.Decimal)) + t, err := types.ParseTime(ctx.GetSessionVars().StmtCtx, vv, col.Tp, col.Decimal) if err != nil { return defaultVal, errors.Trace(err) } @@ -845,7 +930,7 @@ func getDefaultValue(ctx sessionctx.Context, col *table.Column, c *ast.ColumnOpt } } } - vd, err := expression.GetTimeValue(ctx, c.Expr, tp, int8(fsp)) + vd, err := expression.GetTimeValue(ctx, c.Expr, tp, fsp) value := vd.GetValue() if err != nil { return nil, false, ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O) @@ -1294,7 +1379,7 @@ func checkColumnAttributes(colName string, tp *types.FieldType) error { return types.ErrMBiggerThanD.GenWithStackByArgs(colName) } case mysql.TypeDatetime, mysql.TypeDuration, mysql.TypeTimestamp: - if tp.Decimal != int(types.UnspecifiedFsp) && (tp.Decimal < int(types.MinFsp) || tp.Decimal > int(types.MaxFsp)) { + if tp.Decimal != types.UnspecifiedFsp && (tp.Decimal < types.MinFsp || tp.Decimal > types.MaxFsp) { return types.ErrTooBigPrecision.GenWithStackByArgs(tp.Decimal, colName, types.MaxFsp) } } @@ -1628,19 +1713,19 @@ func buildTableInfo( return } -func indexColumnsLen(cols []*model.ColumnInfo, idxCols []*model.IndexColumn) (len int, err error) { +func indexColumnsLen(cols []*model.ColumnInfo, idxCols []*model.IndexColumn) (colLen int, err error) { for _, idxCol := range idxCols { col := model.FindColumnInfo(cols, idxCol.Name.L) if col == nil { err = errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", idxCol.Name.L) return } - var colLen int - colLen, err = getIndexColumnLength(col, idxCol.Length) + var l int + l, err = getIndexColumnLength(col, idxCol.Length) if err != nil { return } - len += colLen + colLen += l } return } @@ -1744,6 +1829,9 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { return err } + if err = checkPartitionColumnsUnique(tbInfo); err != nil { + return err + } switch tbInfo.Partition.Type { case model.PartitionTypeRange: @@ -2194,9 +2282,6 @@ func (d *ddl) BatchCreateTableWithInfo(ctx sessionctx.Context, } // append table job args - if len(job.Args) != 1 { - return errors.Trace(fmt.Errorf("except only one argument")) - } info, ok := job.Args[0].(*model.TableInfo) if !ok { return errors.Trace(fmt.Errorf("except table info")) @@ -2218,8 +2303,8 @@ func (d *ddl) BatchCreateTableWithInfo(ctx sessionctx.Context, return errors.Trace(d.callHookOnChanged(err)) } - for j := range infos { - if err = d.createTableWithInfoPost(ctx, infos[j], jobs.SchemaID); err != nil { + for j := range args { + if err = d.createTableWithInfoPost(ctx, args[j], jobs.SchemaID); err != nil { return errors.Trace(d.callHookOnChanged(err)) } } @@ -4340,6 +4425,7 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Contex return nil, errors.Trace(err) } + tzName, tzOffset := ddlutil.GetTimeZone(sctx) job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, @@ -4350,6 +4436,7 @@ func (d *ddl) getModifiableColumnJob(ctx context.Context, sctx sessionctx.Contex SQLMode: sctx.GetSessionVars().SQLMode, Warnings: make(map[errors.ErrorID]*terror.Error), WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, CtxVars: []interface{}{needChangeColData}, Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp, newAutoRandBits}, @@ -4578,6 +4665,8 @@ func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al } } + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + newCol := oldCol.Clone() newCol.Name = newColName job := &model.Job{ @@ -4590,6 +4679,7 @@ func (d *ddl) RenameColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al SQLMode: ctx.GetSessionVars().SQLMode, Warnings: make(map[errors.ErrorID]*terror.Error), WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, Args: []interface{}{&newCol, oldColName, spec.Position, 0}, } @@ -4643,10 +4733,11 @@ func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt colName := specNewColumn.Name.Name // Check whether alter column has existed. - col := table.FindCol(t.Cols(), colName.L) - if col == nil { + oldCol := table.FindCol(t.Cols(), colName.L) + if oldCol == nil { return ErrBadField.GenWithStackByArgs(colName, ident.Name) } + col := table.ToColumn(oldCol.Clone()) // Clean the NoDefaultValueFlag value. col.Flag &= ^mysql.NoDefaultValueFlag @@ -4782,6 +4873,19 @@ func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Iden return errors.Trace(err) } +func shouldModifyTiFlashReplica(tbReplicaInfo *model.TiFlashReplicaInfo, replicaInfo *ast.TiFlashReplicaSpec) bool { + if tbReplicaInfo != nil && tbReplicaInfo.Count == replicaInfo.Count && + len(tbReplicaInfo.LocationLabels) == len(replicaInfo.Labels) { + for i, label := range tbReplicaInfo.LocationLabels { + if replicaInfo.Labels[i] != label { + return true + } + } + return false + } + return true +} + // AlterTableSetTiFlashReplica sets the TiFlash replicas info. func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Ident, replicaInfo *ast.TiFlashReplicaSpec) error { schema, tb, err := d.getSchemaAndTableByIdent(ctx, ident) @@ -4804,18 +4908,8 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden } tbReplicaInfo := tb.Meta().TiFlashReplica - if tbReplicaInfo != nil && tbReplicaInfo.Count == replicaInfo.Count && - len(tbReplicaInfo.LocationLabels) == len(replicaInfo.Labels) { - changed := false - for i, label := range tbReplicaInfo.LocationLabels { - if replicaInfo.Labels[i] != label { - changed = true - break - } - } - if !changed { - return nil - } + if !shouldModifyTiFlashReplica(tbReplicaInfo, replicaInfo) { + return nil } err = checkTiFlashReplicaCount(ctx, replicaInfo.Count) @@ -5234,7 +5328,7 @@ func extractTblInfos(is infoschema.InfoSchema, oldIdent, newIdent ast.Ident, isA if tableExists(is, newIdent, tables) { return nil, 0, infoschema.ErrTableExists.GenWithStackByArgs(newIdent) } - return nil, 0, errFileNotFound.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) } if !tableExists(is, oldIdent, tables) { if isAlterTable { @@ -5243,7 +5337,7 @@ func extractTblInfos(is infoschema.InfoSchema, oldIdent, newIdent ast.Ident, isA if tableExists(is, newIdent, tables) { return nil, 0, infoschema.ErrTableExists.GenWithStackByArgs(newIdent) } - return nil, 0, errFileNotFound.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) + return nil, 0, infoschema.ErrTableNotExists.GenWithStackByArgs(oldIdent.Schema, oldIdent.Name) } if isAlterTable && newIdent.Schema.L == oldIdent.Schema.L && newIdent.Name.L == oldIdent.Name.L { // oldIdent is equal to newIdent, do nothing @@ -5389,6 +5483,8 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m return errors.Trace(err) } + tzName, tzOffset := ddlutil.GetTimeZone(ctx) + unique := true sqlMode := ctx.GetSessionVars().SQLMode job := &model.Job{ @@ -5401,6 +5497,7 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m SQLMode: ctx.GetSessionVars().SQLMode, Warnings: make(map[errors.ErrorID]*terror.Error), WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, sqlMode, nil, global}, Priority: ctx.GetSessionVars().DDLReorgPriority, @@ -5578,6 +5675,8 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde if _, err = validateCommentLength(ctx.GetSessionVars(), indexName.String(), indexOption); err != nil { return errors.Trace(err) } + + tzName, tzOffset := ddlutil.GetTimeZone(ctx) job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, @@ -5588,6 +5687,7 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde SQLMode: ctx.GetSessionVars().SQLMode, Warnings: make(map[errors.ErrorID]*terror.Error), WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, hiddenCols, global}, Priority: ctx.GetSessionVars().DDLReorgPriority, diff --git a/ddl/ddl_tiflash_api.go b/ddl/ddl_tiflash_api.go index 9070e45a80e80..0678ea304f066 100644 --- a/ddl/ddl_tiflash_api.go +++ b/ddl/ddl_tiflash_api.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" + atomicutil "go.uber.org/atomic" "go.uber.org/zap" ) @@ -53,13 +54,14 @@ type TiFlashReplicaStatus struct { LocationLabels []string Available bool HighPriority bool + IsPartition bool } // TiFlashManagementContext is the context for TiFlash Replica Management type TiFlashManagementContext struct { TiFlashStores map[int64]helper.StoreStat - HandlePdCounter int - UpdateTiFlashStoreCounter int + HandlePdCounter uint64 + UpdateTiFlashStoreCounter uint64 UpdateMap map[int64]bool } @@ -77,9 +79,9 @@ var ( // PollTiFlashInterval is the interval between every pollTiFlashReplicaStatus call. PollTiFlashInterval = 2 * time.Second // PullTiFlashPdTick indicates the number of intervals before we fully sync all TiFlash pd rules and tables. - PullTiFlashPdTick = 30 * 5 + PullTiFlashPdTick = atomicutil.NewUint64(30 * 5) // UpdateTiFlashStoreTick indicates the number of intervals before we fully update TiFlash stores. - UpdateTiFlashStoreTick = 5 + UpdateTiFlashStoreTick = atomicutil.NewUint64(5) ) func getTiflashHTTPAddr(host string, statusAddr string) (string, error) { @@ -131,16 +133,16 @@ func GetTiFlashReplicaInfo(tblInfo *model.TableInfo, tableList *[]TiFlashReplica for _, p := range pi.Definitions { logutil.BgLogger().Debug(fmt.Sprintf("Table %v has partition %v\n", tblInfo.ID, p.ID)) *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, - tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), false}) + tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), false, true}) } // partitions that in adding mid-state for _, p := range pi.AddingDefinitions { - logutil.BgLogger().Debug(fmt.Sprintf("Table %v has partition %v\n", tblInfo.ID, p.ID)) - *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), true}) + logutil.BgLogger().Debug(fmt.Sprintf("Table %v has partition adding %v\n", tblInfo.ID, p.ID)) + *tableList = append(*tableList, TiFlashReplicaStatus{p.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.IsPartitionAvailable(p.ID), true, true}) } } else { logutil.BgLogger().Debug(fmt.Sprintf("Table %v has no partition\n", tblInfo.ID)) - *tableList = append(*tableList, TiFlashReplicaStatus{tblInfo.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.Available, false}) + *tableList = append(*tableList, TiFlashReplicaStatus{tblInfo.ID, tblInfo.TiFlashReplica.Count, tblInfo.TiFlashReplica.LocationLabels, tblInfo.TiFlashReplica.Available, false, false}) } } @@ -186,8 +188,9 @@ func updateTiFlashStores(pollTiFlashContext *TiFlashManagementContext) error { // We need the up-to-date information about TiFlash stores. // Since TiFlash Replica synchronize may happen immediately after new TiFlash stores are added. tikvStats, err := infosync.GetTiFlashStoresStat(context.Background()) + // If MockTiFlash is not set, will issue a MockTiFlashError here. if err != nil { - return errors.Trace(err) + return err } pollTiFlashContext.TiFlashStores = make(map[int64]helper.StoreStat) for _, store := range tikvStats.Stores { @@ -206,19 +209,19 @@ func (d *ddl) pollTiFlashReplicaStatus(ctx sessionctx.Context, pollTiFlashContex allReplicaReady := true defer func() { pollTiFlashContext.HandlePdCounter += 1 - pollTiFlashContext.HandlePdCounter %= PullTiFlashPdTick + pollTiFlashContext.HandlePdCounter %= PullTiFlashPdTick.Load() }() - updateTiFlash := pollTiFlashContext.UpdateTiFlashStoreCounter%UpdateTiFlashStoreTick == 0 + updateTiFlash := pollTiFlashContext.UpdateTiFlashStoreCounter%UpdateTiFlashStoreTick.Load() == 0 if updateTiFlash { if err := updateTiFlashStores(pollTiFlashContext); err != nil { // If we failed to get from pd, retry everytime. pollTiFlashContext.UpdateTiFlashStoreCounter = 0 - return false, errors.Trace(err) + return false, err } } pollTiFlashContext.UpdateTiFlashStoreCounter += 1 - pollTiFlashContext.UpdateTiFlashStoreCounter %= UpdateTiFlashStoreTick + pollTiFlashContext.UpdateTiFlashStoreCounter %= UpdateTiFlashStoreTick.Load() // The following loop updates TiFlash store's status address. for _, store := range pollTiFlashContext.TiFlashStores { @@ -269,11 +272,11 @@ func (d *ddl) pollTiFlashReplicaStatus(ctx sessionctx.Context, pollTiFlashContex } } - logutil.BgLogger().Info("CollectTiFlashStatus", zap.Any("regionReplica", regionReplica), zap.Int64("tb.ID", tb.ID)) + logutil.BgLogger().Debug("CollectTiFlashStatus", zap.Any("regionReplica", regionReplica), zap.Int64("tableID", tb.ID)) var stats helper.PDRegionStats if err := infosync.GetTiFlashPDRegionRecordStats(context.Background(), tb.ID, &stats); err != nil { - return allReplicaReady, errors.Trace(err) + return allReplicaReady, err } regionCount := stats.Count flashRegionCount := len(regionReplica) @@ -283,21 +286,26 @@ func (d *ddl) pollTiFlashReplicaStatus(ctx sessionctx.Context, pollTiFlashContex }) if !avail { - logutil.BgLogger().Info("Tiflash replica is not available", zap.Int64("id", tb.ID), zap.Uint64("region need", uint64(regionCount)), zap.Uint64("region have", uint64(flashRegionCount))) + logutil.BgLogger().Info("Tiflash replica is not available", zap.Int64("tableID", tb.ID), zap.Uint64("region need", uint64(regionCount)), zap.Uint64("region have", uint64(flashRegionCount))) err := infosync.UpdateTiFlashTableSyncProgress(context.Background(), tb.ID, float64(flashRegionCount)/float64(regionCount)) if err != nil { - return false, errors.Trace(err) + return false, err } } else { - logutil.BgLogger().Info("Tiflash replica is available", zap.Int64("id", tb.ID), zap.Uint64("region need", uint64(regionCount))) + logutil.BgLogger().Info("Tiflash replica is available", zap.Int64("tableID", tb.ID), zap.Uint64("region need", uint64(regionCount))) err := infosync.DeleteTiFlashTableSyncProgress(tb.ID) if err != nil { - return false, errors.Trace(err) + return false, err } } // Will call `onUpdateFlashReplicaStatus` to update `TiFlashReplica`. if err := d.UpdateTableReplicaInfo(ctx, tb.ID, avail); err != nil { - logutil.BgLogger().Error("UpdateTableReplicaInfo error when updating TiFlash replica status", zap.Error(err)) + if infoschema.ErrTableNotExists.Equal(err) && tb.IsPartition { + // May be due to blocking add partition + logutil.BgLogger().Info("updating TiFlash replica status err, maybe false alarm by blocking add", zap.Error(err), zap.Int64("tableID", tb.ID), zap.Bool("isPartition", tb.IsPartition)) + } else { + logutil.BgLogger().Error("updating TiFlash replica status err", zap.Error(err), zap.Int64("tableID", tb.ID), zap.Bool("isPartition", tb.IsPartition)) + } } } } @@ -412,6 +420,11 @@ func HandlePlacementRuleRoutine(ctx sessionctx.Context, d *ddl, tableList []TiFl func (d *ddl) PollTiFlashRoutine() { pollTiflashContext := NewTiFlashManagementContext() for { + select { + case <-d.ctx.Done(): + return + case <-time.After(PollTiFlashInterval): + } if d.IsTiFlashPollEnabled() { if d.sessPool == nil { logutil.BgLogger().Error("failed to get sessionPool for pollTiFlashReplicaStatus") @@ -425,7 +438,12 @@ func (d *ddl) PollTiFlashRoutine() { if d.ownerManager.IsOwner() { _, err := d.pollTiFlashReplicaStatus(sctx, pollTiflashContext) if err != nil { - logutil.BgLogger().Warn("pollTiFlashReplicaStatus returns error", zap.Error(err)) + switch err.(type) { + case *infosync.MockTiFlashError: + // If we have not set up MockTiFlash instance, for those tests without TiFlash, just suppress. + default: + logutil.BgLogger().Warn("pollTiFlashReplicaStatus returns error", zap.Error(err)) + } } } d.sessPool.put(sctx) @@ -436,10 +454,5 @@ func (d *ddl) PollTiFlashRoutine() { logutil.BgLogger().Error("failed to get session for pollTiFlashReplicaStatus", zap.Error(err)) } } - select { - case <-d.ctx.Done(): - return - case <-time.After(PollTiFlashInterval): - } } } diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index 7569a0a19c17d..a92223f78c1ad 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -22,12 +22,11 @@ import ( "context" "fmt" "math" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/log" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" @@ -38,28 +37,29 @@ import ( "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" - "go.uber.org/zap" ) -type tiflashDDLTestSuite struct { +type tiflashContext struct { store kv.Storage dom *domain.Domain tiflash *infosync.MockTiFlash cluster *unistore.Cluster } -var _ = SerialSuites(&tiflashDDLTestSuite{}) - const ( RoundToBeAvailable = 2 RoundToBeAvailablePartitionTable = 3 ) -func (s *tiflashDDLTestSuite) SetUpSuite(c *C) { +func createTiFlashContext(t *testing.T) (*tiflashContext, func()) { + s := &tiflashContext{} var err error + ddl.PollTiFlashInterval = 1000 * time.Millisecond + ddl.PullTiFlashPdTick.Store(60) s.tiflash = infosync.NewMockTiFlash() s.store, err = mockstore.NewMockStore( mockstore.WithClusterInspector(func(c testutils.Cluster) { @@ -79,65 +79,23 @@ func (s *tiflashDDLTestSuite) SetUpSuite(c *C) { mockstore.WithStoreType(mockstore.EmbedUnistore), ) - c.Assert(err, IsNil) - + require.NoError(t, err) session.SetSchemaLease(0) session.DisableStats4Test() - s.dom, err = session.BootstrapSession(s.store) - infosync.SetMockTiFlash(s.tiflash) - c.Assert(err, IsNil) + require.NoError(t, err) s.dom.SetStatsUpdating(true) - log.Info("Mock stat", zap.Any("infosyncer", s.dom.InfoSyncer())) - ddl.PollTiFlashInterval = 1000 * time.Millisecond - ddl.PullTiFlashPdTick = 60 -} - -func (s *tiflashDDLTestSuite) TearDownSuite(c *C) { - s.tiflash.StatusServer.Close() - s.dom.Close() - err := s.store.Close() - c.Assert(err, IsNil) - ddl.PollTiFlashInterval = 2 * time.Second -} - -// Compare supposed rule, and we actually get from TableInfo -func isRuleMatch(rule placement.TiFlashRule, startKey string, endKey string, count int, labels []string) bool { - // Compute startKey - if rule.StartKeyHex == startKey && rule.EndKeyHex == endKey { - ok := false - for _, c := range rule.Constraints { - if c.Key == "engine" && len(c.Values) == 1 && c.Values[0] == "tiflash" && c.Op == placement.In { - ok = true - break - } - } - if !ok { - return false - } - - if len(rule.LocationLabels) == len(labels) { - for i, lb := range labels { - if lb != rule.LocationLabels[i] { - return false - } - } - } else { - return false - } - - if rule.Count != count { - return false - } - if rule.Role != placement.Learner { - return false - } - } else { - return false + tearDown := func() { + s.tiflash.Lock() + s.tiflash.StatusServer.Close() + s.tiflash.Unlock() + s.dom.Close() + require.NoError(t, s.store.Close()) + ddl.PollTiFlashInterval = 2 * time.Second } - return true + return s, tearDown } func ChangeGCSafePoint(tk *testkit.TestKit, t time.Time, enable string, lifeTime string) { @@ -160,18 +118,12 @@ func ChangeGCSafePoint(tk *testkit.TestKit, t time.Time, enable string, lifeTime tk.MustExec(s) } -func (s *tiflashDDLTestSuite) CheckPlacementRule(rule placement.TiFlashRule) bool { - for _, r := range s.tiflash.GlobalTiFlashPlacementRules { - if isRuleMatch(rule, r.StartKeyHex, r.EndKeyHex, r.Count, r.LocationLabels) { - return true - } - } - return false +func CheckPlacementRule(tiflash *infosync.MockTiFlash, rule placement.TiFlashRule) bool { + return tiflash.CheckPlacementRule(rule) } -func (s *tiflashDDLTestSuite) CheckFlashback(tk *testkit.TestKit, c *C) { +func (s *tiflashContext) CheckFlashback(tk *testkit.TestKit, t *testing.T) { // If table is dropped after tikv_gc_safe_point, it can be recovered - ChangeGCSafePoint(tk, time.Now().Add(-time.Hour), "false", "10m0s") defer func() { ChangeGCSafePoint(tk, time.Now(), "true", "10m0s") @@ -182,21 +134,21 @@ func (s *tiflashDDLTestSuite) CheckFlashback(tk *testkit.TestKit, c *C) { tk.MustExec("drop table if exists ddltiflash") tk.MustExec("flashback table ddltiflash") time.Sleep(ddl.PollTiFlashInterval * 3) - CheckTableAvailable(s.dom, c, 1, []string{}) + CheckTableAvailable(s.dom, t, 1, []string{}) tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) - c.Assert(tb, NotNil) + require.NoError(t, err) + require.NotNil(t, tb) if tb.Meta().Partition != nil { for _, e := range tb.Meta().Partition.Definitions { ruleName := fmt.Sprintf("table-%v-r", e.ID) - _, ok := s.tiflash.GlobalTiFlashPlacementRules[ruleName] - c.Assert(ok, Equals, true) + _, ok := s.tiflash.GetPlacementRule(ruleName) + require.True(t, ok) } } else { ruleName := fmt.Sprintf("table-%v-r", tb.Meta().ID) - _, ok := s.tiflash.GlobalTiFlashPlacementRules[ruleName] - c.Assert(ok, Equals, true) + _, ok := s.tiflash.GetPlacementRule(ruleName) + require.True(t, ok) } } @@ -213,18 +165,20 @@ func TempDisableEmulatorGC() func() { return f } -func (s *tiflashDDLTestSuite) SetPdLoop(tick int) func() { - originValue := ddl.PullTiFlashPdTick - ddl.PullTiFlashPdTick = tick +func (s *tiflashContext) SetPdLoop(tick uint64) func() { + originValue := ddl.PullTiFlashPdTick.Swap(tick) return func() { - ddl.PullTiFlashPdTick = originValue + ddl.PullTiFlashPdTick.Store(originValue) } } // Run all kinds of DDLs, and will create no redundant pd rules for TiFlash. -func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { +func TestTiFlashNoRedundantPDRules(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + rpcClient, pdClient, cluster, err := unistore.New("") - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { rpcClient.Close() pdClient.Close() @@ -234,13 +188,13 @@ func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { cluster.AddStore(store.Id, store.Address, store.Labels...) } gcWorker, err := gcworker.NewMockGCWorker(s.store) - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, s.store) + require.NoError(t, err) + tk := testkit.NewTestKit(t, s.store) fCancel := TempDisableEmulatorGC() defer fCancel() // Disable emulator GC, otherwise delete range will be automatically called. - failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed", `return`)) defer func() { failpoint.Disable("github.com/pingcap/tidb/store/gcworker/ignoreDeleteRangeFailed") }() @@ -248,7 +202,7 @@ func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { defer fCancelPD() // Clean all rules - s.tiflash.GlobalTiFlashPlacementRules = make(map[string]placement.TiFlashRule) + s.tiflash.CleanPlacementRules() tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("drop table if exists ddltiflashp") @@ -256,68 +210,69 @@ func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { tk.MustExec("create table ddltiflashp(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10),PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (30))") total := 0 - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("alter table ddltiflash set tiflash replica 1") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("alter table ddltiflashp set tiflash replica 1") total += 3 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) lessThan := 40 tk.MustExec(fmt.Sprintf("ALTER TABLE ddltiflashp ADD PARTITION (PARTITION pn VALUES LESS THAN (%v))", lessThan)) total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("alter table ddltiflashp truncate partition p1") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) - + require.Equal(t, total, s.tiflash.PlacementRulesLen()) // Now gc will trigger, and will remove dropped partition. - c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + require.NoError(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("alter table ddltiflashp drop partition p2") - c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + require.NoError(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("truncate table ddltiflash") total += 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) - - c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) + require.NoError(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) tk.MustExec("drop table ddltiflash") total -= 1 time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) - c.Assert(len(s.tiflash.GlobalTiFlashPlacementRules), Equals, total) + require.NoError(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) + require.Equal(t, total, s.tiflash.PlacementRulesLen()) } -func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableNormal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashReplicaPartitionTableNormal(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10),PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (30))") tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) + require.NoError(t, err) replica := tb.Meta().TiFlashReplica - c.Assert(replica, IsNil) + require.Nil(t, replica) tk.MustExec("alter table ddltiflash set tiflash replica 1") lessThan := "40" @@ -325,92 +280,108 @@ func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableNormal(c *C) { time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) // Should get schema again - CheckTableAvailable(s.dom, c, 1, []string{}) + CheckTableAvailable(s.dom, t, 1, []string{}) tb2, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) - c.Assert(tb2, NotNil) + require.NoError(t, err) + require.NotNil(t, tb2) pi := tb2.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) - c.Assert(tb2.Meta().TiFlashReplica, NotNil) + require.NotNil(t, pi) + require.NotNil(t, tb2.Meta().TiFlashReplica) for _, p := range pi.Definitions { - c.Assert(tb2.Meta().TiFlashReplica.IsPartitionAvailable(p.ID), Equals, true) + require.True(t, tb2.Meta().TiFlashReplica.IsPartitionAvailable(p.ID)) if len(p.LessThan) == 1 && p.LessThan[0] == lessThan { - table, ok := s.tiflash.SyncStatus[int(p.ID)] - c.Assert(ok, Equals, true) - c.Assert(table.Accel, Equals, true) + table, ok := s.tiflash.GetTableSyncStatus(int(p.ID)) + require.True(t, ok) + require.True(t, table.Accel) } } - c.Assert(len(pi.AddingDefinitions), Equals, 0) - - s.CheckFlashback(tk, c) + require.Zero(t, len(pi.AddingDefinitions)) + s.CheckFlashback(tk, t) } // When block add partition, new partition shall be available even we break `UpdateTableReplicaInfo` -func (s *tiflashDDLTestSuite) TestTiFlashReplicaPartitionTableBlock(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashReplicaPartitionTableBlock(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int) PARTITION BY RANGE(z) (PARTITION p0 VALUES LESS THAN (10),PARTITION p1 VALUES LESS THAN (20), PARTITION p2 VALUES LESS THAN (30))") tk.MustExec("alter table ddltiflash set tiflash replica 1") // Make sure is available time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - CheckTableAvailable(s.dom, c, 1, []string{}) + CheckTableAvailable(s.dom, t, 1, []string{}) lessThan := "40" // Stop loop - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/BeforePollTiFlashReplicaStatusLoop", `return`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/BeforePollTiFlashReplicaStatusLoop", `return`)) defer func() { _ = failpoint.Disable("github.com/pingcap/tidb/ddl/BeforePollTiFlashReplicaStatusLoop") }() tk.MustExec(fmt.Sprintf("ALTER TABLE ddltiflash ADD PARTITION (PARTITION pn VALUES LESS THAN (%v))", lessThan)) tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) + require.NoError(t, err) pi := tb.Meta().GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) // Partition `lessThan` shall be ready for _, p := range pi.Definitions { - c.Assert(tb.Meta().TiFlashReplica.IsPartitionAvailable(p.ID), Equals, true) + require.True(t, tb.Meta().TiFlashReplica.IsPartitionAvailable(p.ID)) if len(p.LessThan) == 1 && p.LessThan[0] == lessThan { - table, ok := s.tiflash.SyncStatus[int(p.ID)] - c.Assert(ok, Equals, true) - c.Assert(table.Accel, Equals, true) + table, ok := s.tiflash.GetTableSyncStatus(int(p.ID)) + require.True(t, ok) + require.True(t, table.Accel) } } - c.Assert(len(pi.AddingDefinitions), Equals, 0) - s.CheckFlashback(tk, c) + require.Equal(t, 0, len(pi.AddingDefinitions)) + s.CheckFlashback(tk, t) } // TiFlash Table shall be eventually available. -func (s *tiflashDDLTestSuite) TestTiFlashReplicaAvailable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashReplicaAvailable(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int)") tk.MustExec("alter table ddltiflash set tiflash replica 1") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) - CheckTableAvailable(s.dom, c, 1, []string{}) + CheckTableAvailable(s.dom, t, 1, []string{}) tk.MustExec("drop table if exists ddltiflash2") tk.MustExec("create table ddltiflash2 like ddltiflash") tk.MustExec("alter table ddltiflash2 set tiflash replica 1") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 3) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash2") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash2") - s.CheckFlashback(tk, c) + s.CheckFlashback(tk, t) + tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) + require.NoError(t, err) + r, ok := s.tiflash.GetPlacementRule(fmt.Sprintf("table-%v-r", tb.Meta().ID)) + require.NotNil(t, r) + require.True(t, ok) tk.MustExec("alter table ddltiflash set tiflash replica 0") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) + tb, err = s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) + require.NoError(t, err) replica := tb.Meta().TiFlashReplica - c.Assert(replica, IsNil) + require.Nil(t, replica) + r, ok = s.tiflash.GetPlacementRule(fmt.Sprintf("table-%v-r", tb.Meta().ID)) + require.Nil(t, r) + require.False(t, ok) } // Truncate partition shall not block. -func (s *tiflashDDLTestSuite) TestTiFlashTruncatePartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashTruncatePartition(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(i int not null, s varchar(255)) partition by range (i) (partition p0 values less than (10), partition p1 values less than (20))") @@ -419,18 +390,21 @@ func (s *tiflashDDLTestSuite) TestTiFlashTruncatePartition(c *C) { tk.MustExec("insert into ddltiflash values(1, 'abc'), (11, 'def')") tk.MustExec("alter table ddltiflash truncate partition p1") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash") } // Fail truncate partition. -func (s *tiflashDDLTestSuite) TestTiFlashFailTruncatePartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashFailTruncatePartition(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(i int not null, s varchar(255)) partition by range (i) (partition p0 values less than (10), partition p1 values less than (20))") tk.MustExec("alter table ddltiflash set tiflash replica 1") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/FailTiFlashTruncatePartition", `return`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/FailTiFlashTruncatePartition", `return`)) defer func() { failpoint.Disable("github.com/pingcap/tidb/ddl/FailTiFlashTruncatePartition") }() @@ -439,40 +413,46 @@ func (s *tiflashDDLTestSuite) TestTiFlashFailTruncatePartition(c *C) { tk.MustExec("insert into ddltiflash values(1, 'abc'), (11, 'def')") tk.MustGetErrMsg("alter table ddltiflash truncate partition p1", "[ddl:-1]enforced error") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash") } // Drop partition shall not block. -func (s *tiflashDDLTestSuite) TestTiFlashDropPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashDropPartition(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(i int not null, s varchar(255)) partition by range (i) (partition p0 values less than (10), partition p1 values less than (20))") tk.MustExec("alter table ddltiflash set tiflash replica 1") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash") tk.MustExec("alter table ddltiflash drop partition p1") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable * 5) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash") } -func CheckTableAvailableWithTableName(dom *domain.Domain, c *C, count uint64, labels []string, db string, table string) { +func CheckTableAvailableWithTableName(dom *domain.Domain, t *testing.T, count uint64, labels []string, db string, table string) { tb, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) - c.Assert(err, IsNil) + require.NoError(t, err) replica := tb.Meta().TiFlashReplica - c.Assert(replica, NotNil) - c.Assert(replica.Available, Equals, true) - c.Assert(replica.Count, Equals, count) - c.Assert(replica.LocationLabels, DeepEquals, labels) + require.NotNil(t, replica) + require.True(t, replica.Available) + require.Equal(t, count, replica.Count) + require.ElementsMatch(t, labels, replica.LocationLabels) } -func CheckTableAvailable(dom *domain.Domain, c *C, count uint64, labels []string) { - CheckTableAvailableWithTableName(dom, c, count, labels, "test", "ddltiflash") +func CheckTableAvailable(dom *domain.Domain, t *testing.T, count uint64, labels []string) { + CheckTableAvailableWithTableName(dom, t, count, labels, "test", "ddltiflash") } // Truncate table shall not block. -func (s *tiflashDDLTestSuite) TestTiFlashTruncateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTiFlashTruncateTable(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflashp") tk.MustExec("create table ddltiflashp(z int not null) partition by range (z) (partition p0 values less than (10), partition p1 values less than (20))") @@ -482,7 +462,7 @@ func (s *tiflashDDLTestSuite) TestTiFlashTruncateTable(c *C) { // Should get schema right now tk.MustExec("truncate table ddltiflashp") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailablePartitionTable) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflashp") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflashp") tk.MustExec("drop table if exists ddltiflash2") tk.MustExec("create table ddltiflash2(z int)") tk.MustExec("alter table ddltiflash2 set tiflash replica 1") @@ -491,14 +471,16 @@ func (s *tiflashDDLTestSuite) TestTiFlashTruncateTable(c *C) { tk.MustExec("truncate table ddltiflash2") time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", "ddltiflash2") + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", "ddltiflash2") } // TiFlash Table shall be eventually available, even with lots of small table created. -func (s *tiflashDDLTestSuite) TestTiFlashMassiveReplicaAvailable(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") +func TestTiFlashMassiveReplicaAvailable(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("drop table if exists ddltiflash%v", i)) tk.MustExec(fmt.Sprintf("create table ddltiflash%v(z int)", i)) @@ -508,23 +490,26 @@ func (s *tiflashDDLTestSuite) TestTiFlashMassiveReplicaAvailable(c *C) { time.Sleep(ddl.PollTiFlashInterval * 10) // Should get schema right now for i := 0; i < 100; i++ { - CheckTableAvailableWithTableName(s.dom, c, 1, []string{}, "test", fmt.Sprintf("ddltiflash%v", i)) + CheckTableAvailableWithTableName(s.dom, t, 1, []string{}, "test", fmt.Sprintf("ddltiflash%v", i)) } } // When set TiFlash replica, tidb shall add one Pd Rule for this table. // When drop/truncate table, Pd Rule shall be removed in limited time. -func (s *tiflashDDLTestSuite) TestSetPlacementRuleNormal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetPlacementRuleNormal(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int)") tk.MustExec("alter table ddltiflash set tiflash replica 1 location labels 'a','b'") tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) + require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{"a", "b"}) - res := s.CheckPlacementRule(*expectRule) - c.Assert(res, Equals, true) + res := CheckPlacementRule(s.tiflash, *expectRule) + require.True(t, res) // Set lastSafePoint to a timepoint in future, so all dropped table can be reckon as gc-ed. ChangeGCSafePoint(tk, time.Now().Add(+3*time.Second), "true", "10m0s") @@ -535,14 +520,17 @@ func (s *tiflashDDLTestSuite) TestSetPlacementRuleNormal(c *C) { defer fCancelPD() tk.MustExec("drop table ddltiflash") expectRule = infosync.MakeNewRule(tb.Meta().ID, 1, []string{"a", "b"}) - res = s.CheckPlacementRule(*expectRule) - c.Assert(res, Equals, true) + res = CheckPlacementRule(s.tiflash, *expectRule) + require.True(t, res) } // When gc worker works, it will automatically remove pd rule for TiFlash. -func (s *tiflashDDLTestSuite) TestSetPlacementRuleWithGCWorker(c *C) { + +func TestSetPlacementRuleWithGCWorker(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + rpcClient, pdClient, cluster, err := unistore.New("") - c.Assert(err, IsNil) defer func() { rpcClient.Close() pdClient.Close() @@ -557,48 +545,107 @@ func (s *tiflashDDLTestSuite) TestSetPlacementRuleWithGCWorker(c *C) { }() fCancelPD := s.SetPdLoop(10000) defer fCancelPD() + + require.NoError(t, err) gcWorker, err := gcworker.NewMockGCWorker(s.store) - c.Assert(err, IsNil) + require.NoError(t, err) // Make SetPdLoop take effects. time.Sleep(time.Second) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash_gc") tk.MustExec("create table ddltiflash_gc(z int)") tk.MustExec("alter table ddltiflash_gc set tiflash replica 1 location labels 'a','b'") tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash_gc")) - c.Assert(err, IsNil) + require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{"a", "b"}) - res := s.CheckPlacementRule(*expectRule) - c.Assert(res, Equals, true) + res := CheckPlacementRule(s.tiflash, *expectRule) + require.True(t, res) ChangeGCSafePoint(tk, time.Now().Add(-time.Hour), "true", "10m0s") tk.MustExec("drop table ddltiflash_gc") // Now gc will trigger, and will remove dropped table. - c.Assert(gcWorker.DeleteRanges(context.TODO(), math.MaxInt64), IsNil) + require.Nil(t, gcWorker.DeleteRanges(context.TODO(), math.MaxInt64)) // Wait GC time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable) - res = s.CheckPlacementRule(*expectRule) - c.Assert(res, Equals, false) + res = CheckPlacementRule(s.tiflash, *expectRule) + require.False(t, res) } -func (s *tiflashDDLTestSuite) TestSetPlacementRuleFail(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetPlacementRuleFail(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + tk.MustExec("use test") tk.MustExec("drop table if exists ddltiflash") tk.MustExec("create table ddltiflash(z int)") - s.tiflash.PdEnabled = false + s.tiflash.PdSwitch(false) defer func() { - s.tiflash.PdEnabled = true + s.tiflash.PdSwitch(true) }() tk.MustExec("alter table ddltiflash set tiflash replica 1") tb, err := s.dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("ddltiflash")) - c.Assert(err, IsNil) + require.NoError(t, err) expectRule := infosync.MakeNewRule(tb.Meta().ID, 1, []string{}) - res := s.CheckPlacementRule(*expectRule) - c.Assert(res, Equals, false) + res := CheckPlacementRule(s.tiflash, *expectRule) + require.False(t, res) +} + +func TestAlterDatabaseErrorGrammar(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + }() + session.SetSchemaLease(0) + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + defer dom.Close() + + tk := testkit.NewTestKit(t, store) + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change") +} + +func TestAlterDatabaseBasic(t *testing.T) { + s, teardown := createTiFlashContext(t) + defer teardown() + tk := testkit.NewTestKit(t, s.store) + + tk.MustExec("drop database if exists tiflash_ddl") + tk.MustExec("create database tiflash_ddl") + tk.MustExec("create table tiflash_ddl.ddltiflash(z int)") + tk.MustExec("create table tiflash_ddl.ddltiflash2(z int)") + // ALTER DATABASE can override previous ALTER TABLE. + tk.MustExec("alter table tiflash_ddl.ddltiflash set tiflash replica 1") + tk.MustExec("alter database tiflash_ddl set tiflash replica 2") + require.Equal(t, "In total 2 tables: 2 succeed, 0 failed, 0 skipped", tk.Session().GetSessionVars().StmtCtx.GetMessage()) + time.Sleep(ddl.PollTiFlashInterval * RoundToBeAvailable * 2) + CheckTableAvailableWithTableName(s.dom, t, 2, []string{}, "tiflash_ddl", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 2, []string{}, "tiflash_ddl", "ddltiflash2") + + // Skip already set TiFlash tables. + tk.MustExec("alter database tiflash_ddl set tiflash replica 2") + require.Equal(t, "In total 2 tables: 0 succeed, 0 failed, 2 skipped", tk.Session().GetSessionVars().StmtCtx.GetMessage()) + CheckTableAvailableWithTableName(s.dom, t, 2, []string{}, "tiflash_ddl", "ddltiflash") + CheckTableAvailableWithTableName(s.dom, t, 2, []string{}, "tiflash_ddl", "ddltiflash2") + + // There is no existing database. + tk.MustExec("drop database if exists tiflash_ddl_missing") + tk.MustGetErrMsg("alter database tiflash_ddl_missing set tiflash replica 2", "[schema:1049]Unknown database 'tiflash_ddl_missing'") + + // There is no table in database + tk.MustExec("drop database if exists tiflash_ddl_empty") + tk.MustExec("create database tiflash_ddl_empty") + tk.MustGetErrMsg("alter database tiflash_ddl_empty set tiflash replica 2", "[schema:1049]Empty database 'tiflash_ddl_empty'") + + // There is less TiFlash store + tk.MustGetErrMsg("alter database tiflash_ddl set tiflash replica 3", "the tiflash replica count: 3 should be less than the total tiflash server count: 2") } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index e69893026cfff..ae0348efd2372 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -92,7 +92,9 @@ type worker struct { reorgCtx *reorgCtx // reorgCtx is used for reorganization. delRangeManager delRangeManager logCtx context.Context + lockSeqNum bool + *ddlCtx ddlJobCache } @@ -105,7 +107,7 @@ type ddlJobCache struct { cacheDigest *parser.Digest } -func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRangeMgr delRangeManager) *worker { +func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRangeMgr delRangeManager, dCtx *ddlCtx) *worker { worker := &worker{ id: atomic.AddInt32(&ddlWorkerID, 1), tp: tp, @@ -117,6 +119,7 @@ func newWorker(ctx context.Context, tp workerType, sessPool *sessionPool, delRan cacheNormalizedSQL: "", cacheDigest: nil, }, + ddlCtx: dCtx, reorgCtx: &reorgCtx{notifyCancelReorgJob: 0}, sessPool: sessPool, delRangeManager: delRangeMgr, @@ -440,10 +443,18 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { // Notice: warnings is used to support non-strict mode. updateRawArgs = false } + w.writeDDLSeqNum(job) err = t.AddHistoryDDLJob(job, updateRawArgs) return errors.Trace(err) } +func (w *worker) writeDDLSeqNum(job *model.Job) { + w.ddlSeqNumMu.Lock() + w.ddlSeqNumMu.seqNum++ + w.lockSeqNum = true + job.SeqNum = w.ddlSeqNumMu.seqNum +} + func finishRecoverTable(w *worker, job *model.Job) error { tbInfo := &model.TableInfo{} var autoIncID, autoRandID, dropJobID, recoverTableCheckFlag int64 @@ -592,11 +603,21 @@ func (w *worker) handleDDLJobQueue(d *ddlCtx) error { } if err != nil { + if w.lockSeqNum { + // txn commit failed, we should reset seqNum. + w.ddlSeqNumMu.seqNum-- + w.lockSeqNum = false + w.ddlSeqNumMu.Unlock() + } return errors.Trace(err) } else if job == nil { // No job now, return and retry getting later. return nil } + if w.lockSeqNum { + w.lockSeqNum = false + d.ddlSeqNumMu.Unlock() + } w.waitDependencyJobFinished(job, &waitDependencyJobCnt) // Here means the job enters another state (delete only, write only, public, etc...) or is cancelled. @@ -736,6 +757,9 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, logutil.Logger(w.logCtx).Info("[ddl] run DDL job", zap.String("job", job.String())) timeStart := time.Now() + if job.RealStartTS == 0 { + job.RealStartTS = t.StartTS + } defer func() { metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerRunDDLJob, job.Type.String(), metrics.RetLabel(err)).Observe(time.Since(timeStart).Seconds()) }() @@ -774,7 +798,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionCreateView: ver, err = onCreateView(d, t, job) case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: - ver, err = onDropTableOrView(d, t, job) + ver, err = onDropTableOrView(t, job) case model.ActionDropTablePartition: ver, err = w.onDropTablePartition(d, t, job) case model.ActionTruncateTablePartition: diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 3811b3cc96a36..72dd6133934b4 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -646,7 +646,7 @@ func doDDLJobSuccess(ctx sessionctx.Context, d *ddl, t *testing.T, schemaID, tab BinlogInfo: &model.HistoryInfo{}, } err := d.doDDLJob(ctx, job) - require.Nil(t, err) + require.NoError(t, err) } func doDDLJobErr(t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}, diff --git a/ddl/error.go b/ddl/error.go index 955cfbdbfa589..9db351199aa22 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -30,7 +30,6 @@ var ( errCantDecodeRecord = dbterror.ClassDDL.NewStd(mysql.ErrCantDecodeRecord) errInvalidDDLJob = dbterror.ClassDDL.NewStd(mysql.ErrInvalidDDLJob) errCancelledDDLJob = dbterror.ClassDDL.NewStd(mysql.ErrCancelledDDLJob) - errFileNotFound = dbterror.ClassDDL.NewStd(mysql.ErrFileNotFound) errRunMultiSchemaChanges = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "multi schema change"), nil)) errWaitReorgTimeout = dbterror.ClassDDL.NewStdErr(mysql.ErrLockWaitTimeout, mysql.MySQLErrName[mysql.ErrWaitReorgTimeout]) errInvalidStoreVer = dbterror.ClassDDL.NewStd(mysql.ErrInvalidStoreVersion) @@ -146,6 +145,8 @@ var ( ErrWrongTableName = dbterror.ClassDDL.NewStd(mysql.ErrWrongTableName) // ErrWrongColumnName returns for wrong column name. ErrWrongColumnName = dbterror.ClassDDL.NewStd(mysql.ErrWrongColumnName) + // ErrWrongPartitionName returns for wrong partition name. + ErrWrongPartitionName = dbterror.ClassDDL.NewStd(mysql.ErrWrongPartitionName) // ErrWrongUsage returns for wrong ddl syntax usage. ErrWrongUsage = dbterror.ClassDDL.NewStd(mysql.ErrWrongUsage) // ErrInvalidGroupFuncUse returns for using invalid group functions. @@ -177,6 +178,8 @@ var ( ErrDropPartitionNonExistent = dbterror.ClassDDL.NewStd(mysql.ErrDropPartitionNonExistent) // ErrSameNamePartition returns duplicate partition name. ErrSameNamePartition = dbterror.ClassDDL.NewStd(mysql.ErrSameNamePartition) + // ErrSameNamePartitionField returns duplicate partition field. + ErrSameNamePartitionField = dbterror.ClassDDL.NewStd(mysql.ErrSameNamePartitionField) // ErrRangeNotIncreasing returns values less than value must be strictly increasing for each partition. ErrRangeNotIncreasing = dbterror.ClassDDL.NewStd(mysql.ErrRangeNotIncreasing) // ErrPartitionMaxvalue returns maxvalue can only be used in last partition definition. diff --git a/ddl/failtest/main_test.go b/ddl/failtest/main_test.go index aea99a6687c23..91f147bd7725a 100644 --- a/ddl/failtest/main_test.go +++ b/ddl/failtest/main_test.go @@ -35,7 +35,7 @@ func TestMain(m *testing.M) { ddl.SetWaitTimeWhenErrorOccurred(time.Microsecond) opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/ddl/index.go b/ddl/index.go index 485f0fd1c0856..3381607d8cefa 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" decoder "github.com/pingcap/tidb/util/rowDecoder" - "github.com/pingcap/tidb/util/timeutil" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" @@ -1060,7 +1059,6 @@ var mockNotOwnerErrOnce uint32 // getIndexRecord gets index columns values use w.rowDecoder, and generate indexRecord. func (w *baseIndexWorker) getIndexRecord(idxInfo *model.IndexInfo, handle kv.Handle, recordKey []byte) (*indexRecord, error) { cols := w.table.WritableCols() - sysZone := timeutil.SystemLocation() failpoint.Inject("MockGetIndexRecordErr", func(val failpoint.Value) { if valStr, ok := val.(string); ok { switch valStr { @@ -1094,16 +1092,6 @@ func (w *baseIndexWorker) getIndexRecord(idxInfo *model.IndexInfo, handle kv.Han return nil, errors.Trace(err) } - if idxColumnVal.Kind() == types.KindMysqlTime { - t := idxColumnVal.GetMysqlTime() - if t.Type() == mysql.TypeTimestamp && sysZone != time.UTC { - err := t.ConvertTimeZone(sysZone, time.UTC) - if err != nil { - return nil, errors.Trace(err) - } - idxColumnVal.SetMysqlTime(t) - } - } idxVal[j] = idxColumnVal } @@ -1129,9 +1117,9 @@ func (w *baseIndexWorker) getNextKey(taskRange reorgBackfillTask, taskDone bool) return taskRange.endKey.Next() } -func (w *baseIndexWorker) updateRowDecoder(handle kv.Handle, recordKey []byte, rawRecord []byte) error { - sysZone := timeutil.SystemLocation() - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap) +func (w *baseIndexWorker) updateRowDecoder(handle kv.Handle, rawRecord []byte) error { + sysZone := w.sessCtx.GetSessionVars().StmtCtx.TimeZone + _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, sysZone, w.rowMap) if err != nil { return errors.Trace(errCantDecodeRecord.GenWithStackByArgs("index", err)) } @@ -1165,7 +1153,7 @@ func (w *baseIndexWorker) fetchRowColVals(txn kv.Transaction, taskRange reorgBac } // Decode one row, generate records of this row. - err := w.updateRowDecoder(handle, recordKey, rawRow) + err := w.updateRowDecoder(handle, rawRow) if err != nil { return false, err } @@ -1337,7 +1325,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC } // Create the index. - handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, idxRecord.handle, idxRecord.rsData) + handle, err := w.index.Create(w.sessCtx, txn, idxRecord.vals, idxRecord.handle, idxRecord.rsData, table.WithIgnoreAssertion) if err != nil { if kv.ErrKeyExists.Equal(err) && idxRecord.handle.Equal(handle) { // Index already exists, skip it. diff --git a/ddl/integration_test.go b/ddl/integration_test.go index 789d6e61aca40..1a2213df0d661 100644 --- a/ddl/integration_test.go +++ b/ddl/integration_test.go @@ -19,7 +19,6 @@ import ( "testing" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -85,12 +84,11 @@ func TestDefaultValueInEnum(t *testing.T) { } func TestDDLStatementsBackFill(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") needReorg := false - dom := domain.GetDomain(tk.Session()) dom.DDL().SetHook(&ddl.TestDDLCallback{ Do: dom, OnJobUpdatedExported: func(job *model.Job) { @@ -119,3 +117,17 @@ func TestDDLStatementsBackFill(t *testing.T) { require.Equal(t, tc.expectedNeedReorg, needReorg, tc) } } + +func TestSchema(t *testing.T) { + _, clean := testkit.CreateMockStore(t) + defer clean() + + ddl.ExportTestSchema(t) +} + +func TestTestSerialStatSuite(t *testing.T) { + _, clean := testkit.CreateMockStore(t) + defer clean() + + ddl.ExportTestSerialStatSuite(t) +} diff --git a/ddl/main_test.go b/ddl/main_test.go index 9cd5e3f8f3d21..203638659f784 100644 --- a/ddl/main_test.go +++ b/ddl/main_test.go @@ -27,7 +27,7 @@ func TestMain(m *testing.M) { tikv.EnableFailpoints() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/ddl/mock.go b/ddl/mock.go index 48372be8641f7..7f470cc979a7f 100644 --- a/ddl/mock.go +++ b/ddl/mock.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) var _ util.SchemaSyncer = &MockSchemaSyncer{} diff --git a/ddl/options.go b/ddl/options.go index 980412e41fa2e..e4c59e5d3b5b7 100644 --- a/ddl/options.go +++ b/ddl/options.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) // Option represents an option to initialize the DDL module diff --git a/ddl/options_test.go b/ddl/options_test.go index ed9b9b89e087d..ccd8471c04250 100644 --- a/ddl/options_test.go +++ b/ddl/options_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) func TestOptions(t *testing.T) { diff --git a/ddl/partition.go b/ddl/partition.go index 7bfa3f250c074..2042928700839 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1576,11 +1576,7 @@ func checkExchangePartitionRecordValidation(w *worker, pt *model.TableInfo, inde } defer w.sessPool.put(ctx) - stmt, err := ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(w.ddlJobCtx, true, sql, paramList...) - if err != nil { - return errors.Trace(err) - } - rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(w.ddlJobCtx, stmt) + rows, _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(w.ddlJobCtx, nil, sql, paramList...) if err != nil { return errors.Trace(err) } @@ -1683,6 +1679,20 @@ func checkAddPartitionOnTemporaryMode(tbInfo *model.TableInfo) error { return nil } +func checkPartitionColumnsUnique(tbInfo *model.TableInfo) error { + if len(tbInfo.Partition.Columns) <= 1 { + return nil + } + var columnsMap = make(map[string]struct{}) + for _, col := range tbInfo.Partition.Columns { + if _, ok := columnsMap[col.L]; ok { + return ErrSameNamePartitionField.GenWithStackByArgs(col.L) + } + columnsMap[col.L] = struct{}{} + } + return nil +} + func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { return ast.ErrNoParts.GenWithStackByArgs("partitions") diff --git a/ddl/placement/bundle_test.go b/ddl/placement/bundle_test.go index de02ca6cd51eb..b59f349609f87 100644 --- a/ddl/placement/bundle_test.go +++ b/ddl/placement/bundle_test.go @@ -16,48 +16,45 @@ package placement import ( "encoding/hex" - "errors" + "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testBundleSuite{}) - -type testBundleSuite struct{} - -func (s *testBundleSuite) TestEmpty(c *C) { +func TestEmpty(t *testing.T) { bundle := &Bundle{ID: GroupID(1)} - c.Assert(bundle.IsEmpty(), IsTrue) + require.True(t, bundle.IsEmpty()) bundle = &Bundle{ID: GroupID(1), Index: 1} - c.Assert(bundle.IsEmpty(), IsFalse) + require.False(t, bundle.IsEmpty()) bundle = &Bundle{ID: GroupID(1), Override: true} - c.Assert(bundle.IsEmpty(), IsFalse) + require.False(t, bundle.IsEmpty()) bundle = &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} - c.Assert(bundle.IsEmpty(), IsFalse) + require.False(t, bundle.IsEmpty()) bundle = &Bundle{ID: GroupID(1), Index: 1, Override: true} - c.Assert(bundle.IsEmpty(), IsFalse) + require.False(t, bundle.IsEmpty()) } -func (s *testBundleSuite) TestClone(c *C) { +func TestCloneBundle(t *testing.T) { bundle := &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}} newBundle := bundle.Clone() newBundle.ID = GroupID(2) newBundle.Rules[0] = &Rule{ID: "121"} - c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) - c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) + require.Equal(t, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}, bundle) + require.Equal(t, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}, newBundle) } -func (s *testBundleSuite) TestObjectID(c *C) { +func TestObjectID(t *testing.T) { type TestCase struct { name string bundleID string @@ -72,20 +69,19 @@ func (s *testBundleSuite) TestObjectID(c *C) { {"id of negatives", "TiDB_DDL_-10", 0, ErrInvalidBundleID}, {"id of positive integer", "TiDB_DDL_10", 10, nil}, } - for _, t := range tests { - comment := Commentf("%s", t.name) - bundle := Bundle{ID: t.bundleID} + for _, test := range tests { + bundle := Bundle{ID: test.bundleID} id, err := bundle.ObjectID() - if t.err == nil { - c.Assert(err, IsNil, comment) - c.Assert(id, Equals, t.expectedID, comment) + if test.err == nil { + require.NoError(t, err, test.name) + require.Equal(t, test.expectedID, id, test.name) } else { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + require.ErrorIs(t, err, test.err, test.name) } } } -func (s *testBundleSuite) TestGetLeaderDCByBundle(c *C) { +func TestGetLeaderDCByBundle(t *testing.T) { testcases := []struct { name string bundle *Bundle @@ -330,49 +326,48 @@ func (s *testBundleSuite) TestGetLeaderDCByBundle(c *C) { }, } for _, testcase := range testcases { - comment := Commentf("%s", testcase.name) result, ok := testcase.bundle.GetLeaderDC("zone") if len(testcase.expectedDC) > 0 { - c.Assert(ok, Equals, true, comment) + require.True(t, ok, testcase.name) } else { - c.Assert(ok, Equals, false, comment) + require.False(t, ok, testcase.name) } - c.Assert(result, Equals, testcase.expectedDC, comment) + require.Equal(t, testcase.expectedDC, result, testcase.name) } } -func (s *testBundleSuite) TestString(c *C) { +func TestString(t *testing.T) { bundle := &Bundle{ ID: GroupID(1), } rules1, err := NewRules(Voter, 3, `["+zone=sh", "+zone=sh"]`) - c.Assert(err, IsNil) + require.NoError(t, err) rules2, err := NewRules(Voter, 4, `["-zone=sh", "+zone=bj"]`) - c.Assert(err, IsNil) + require.NoError(t, err) bundle.Rules = append(rules1, rules2...) - c.Assert(bundle.String(), Equals, "{\"group_id\":\"TiDB_DDL_1\",\"group_index\":0,\"group_override\":false,\"rules\":[{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"count\":3,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"sh\"]}]},{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"count\":4,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"notIn\",\"values\":[\"sh\"]},{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"bj\"]}]}]}") + require.Equal(t, "{\"group_id\":\"TiDB_DDL_1\",\"group_index\":0,\"group_override\":false,\"rules\":[{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"count\":3,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"sh\"]}]},{\"group_id\":\"\",\"id\":\"\",\"start_key\":\"\",\"end_key\":\"\",\"role\":\"voter\",\"count\":4,\"label_constraints\":[{\"key\":\"zone\",\"op\":\"notIn\",\"values\":[\"sh\"]},{\"key\":\"zone\",\"op\":\"in\",\"values\":[\"bj\"]}]}]}", bundle.String()) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/placement/MockMarshalFailure")) }() - c.Assert(bundle.String(), Equals, "") + require.Equal(t, "", bundle.String()) } -func (s *testBundleSuite) TestNew(c *C) { - c.Assert(NewBundle(3), DeepEquals, &Bundle{ID: GroupID(3)}) - c.Assert(NewBundle(-1), DeepEquals, &Bundle{ID: GroupID(-1)}) +func TestNewBundle(t *testing.T) { + require.Equal(t, &Bundle{ID: GroupID(3)}, NewBundle(3)) + require.Equal(t, &Bundle{ID: GroupID(-1)}, NewBundle(-1)) _, err := NewBundleFromConstraintsOptions(nil) - c.Assert(err, NotNil) + require.Error(t, err) _, err = NewBundleFromSugarOptions(nil) - c.Assert(err, NotNil) + require.Error(t, err) _, err = NewBundleFromOptions(nil) - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testBundleSuite) TestNewBundleFromOptions(c *C) { +func TestNewBundleFromOptions(t *testing.T) { type TestCase struct { name string input *model.PlacementSettings @@ -706,42 +701,42 @@ func (s *testBundleSuite) TestNewBundleFromOptions(c *C) { err: ErrInvalidConstraintsRelicas, }) - for _, t := range tests { - bundle, err := newBundleFromOptions(t.input) - comment := Commentf("[%s]\nerr1 %s\nerr2 %s", t.name, err, t.err) - if t.err != nil { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + for _, test := range tests { + bundle, err := newBundleFromOptions(test.input) + comment := fmt.Sprintf("[%s]\nerr1 %s\nerr2 %s", test.name, err, test.err) + if test.err != nil { + require.ErrorIs(t, err, test.err, comment) } else { - c.Assert(err, IsNil, comment) - matchRules(t.output, bundle.Rules, comment.CheckCommentString(), c) + require.NoError(t, err, comment) + matchRules(test.output, bundle.Rules, comment, t) } } } -func (s *testBundleSuite) TestResetBundleWithSingleRule(c *C) { +func TestResetBundleWithSingleRule(t *testing.T) { bundle := &Bundle{ ID: GroupID(1), } rules, err := NewRules(Voter, 3, `["+zone=sh", "+zone=sh"]`) - c.Assert(err, IsNil) + require.NoError(t, err) bundle.Rules = rules bundle.Reset(RuleIndexTable, []int64{3}) - c.Assert(bundle.ID, Equals, GroupID(3)) - c.Assert(bundle.Override, Equals, true) - c.Assert(bundle.Index, Equals, RuleIndexTable) - c.Assert(bundle.Rules, HasLen, 1) - c.Assert(bundle.Rules[0].GroupID, Equals, bundle.ID) + require.Equal(t, GroupID(3), bundle.ID) + require.Equal(t, true, bundle.Override) + require.Equal(t, RuleIndexTable, bundle.Index) + require.Len(t, bundle.Rules, 1) + require.Equal(t, bundle.ID, bundle.Rules[0].GroupID) startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(3))) - c.Assert(bundle.Rules[0].StartKeyHex, Equals, startKey) + require.Equal(t, startKey, bundle.Rules[0].StartKeyHex) endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(4))) - c.Assert(bundle.Rules[0].EndKeyHex, Equals, endKey) + require.Equal(t, endKey, bundle.Rules[0].EndKeyHex) } -func (s *testBundleSuite) TestResetBundleWithMultiRules(c *C) { +func TestResetBundleWithMultiRules(t *testing.T) { // build a bundle with three rules. bundle, err := NewBundleFromOptions(&model.PlacementSettings{ LeaderConstraints: `["+zone=bj"]`, @@ -751,129 +746,129 @@ func (s *testBundleSuite) TestResetBundleWithMultiRules(c *C) { LearnerConstraints: `["+zone=cd"]`, Constraints: `["+disk=ssd"]`, }) - c.Assert(err, IsNil) - c.Assert(len(bundle.Rules), Equals, 3) + require.NoError(t, err) + require.Equal(t, 3, len(bundle.Rules)) // test if all the three rules are basic rules even the start key are not set. bundle.Reset(RuleIndexTable, []int64{1, 2, 3}) - c.Assert(bundle.ID, Equals, GroupID(1)) - c.Assert(bundle.Index, Equals, RuleIndexTable) - c.Assert(bundle.Override, Equals, true) - c.Assert(len(bundle.Rules), Equals, 3*3) + require.Equal(t, GroupID(1), bundle.ID) + require.Equal(t, RuleIndexTable, bundle.Index) + require.Equal(t, true, bundle.Override) + require.Equal(t, 3*3, len(bundle.Rules)) // for id 1. startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(1))) endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(2))) - c.Assert(bundle.Rules[0].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[0].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[1].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[1].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[2].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[2].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[0].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[0].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[1].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[1].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[2].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[2].EndKeyHex) // for id 2. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(2))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(3))) - c.Assert(bundle.Rules[3].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[3].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[4].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[4].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[5].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[5].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[3].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[3].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[4].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[4].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[5].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[5].EndKeyHex) // for id 3. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(3))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(4))) - c.Assert(bundle.Rules[6].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[6].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[7].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[7].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[8].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[8].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[6].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[6].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[7].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[7].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[8].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[8].EndKeyHex) // test if bundle has redundant rules. // for now, the bundle has 9 rules, each table id or partition id has the three with them. // once we reset this bundle for another ids, for example, adding partitions. we should // extend the basic rules(3 of them) to the new partition id. bundle.Reset(RuleIndexTable, []int64{1, 3, 4, 5}) - c.Assert(bundle.ID, Equals, GroupID(1)) - c.Assert(bundle.Index, Equals, RuleIndexTable) - c.Assert(bundle.Override, Equals, true) - c.Assert(len(bundle.Rules), Equals, 3*4) + require.Equal(t, GroupID(1), bundle.ID) + require.Equal(t, RuleIndexTable, bundle.Index) + require.Equal(t, true, bundle.Override) + require.Equal(t, 3*4, len(bundle.Rules)) // for id 1. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(1))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(2))) - c.Assert(bundle.Rules[0].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[0].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[1].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[1].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[2].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[2].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[0].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[0].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[1].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[1].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[2].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[2].EndKeyHex) // for id 3. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(3))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(4))) - c.Assert(bundle.Rules[3].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[3].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[4].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[4].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[5].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[5].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[3].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[3].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[4].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[4].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[5].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[5].EndKeyHex) // for id 4. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(4))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(5))) - c.Assert(bundle.Rules[6].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[6].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[7].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[7].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[8].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[8].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[6].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[6].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[7].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[7].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[8].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[8].EndKeyHex) // for id 5. startKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(5))) endKey = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(6))) - c.Assert(bundle.Rules[9].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[9].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[10].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[10].EndKeyHex, Equals, endKey) - c.Assert(bundle.Rules[11].StartKeyHex, Equals, startKey) - c.Assert(bundle.Rules[11].EndKeyHex, Equals, endKey) + require.Equal(t, startKey, bundle.Rules[9].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[9].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[10].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[10].EndKeyHex) + require.Equal(t, startKey, bundle.Rules[11].StartKeyHex) + require.Equal(t, endKey, bundle.Rules[11].EndKeyHex) } -func (s *testBundleSuite) TestTidy(c *C) { +func TestTidy(t *testing.T) { bundle := &Bundle{ ID: GroupID(1), } rules0, err := NewRules(Voter, 1, `["+zone=sh", "+zone=sh"]`) - c.Assert(err, IsNil) - c.Assert(rules0, HasLen, 1) + require.NoError(t, err) + require.Len(t, rules0, 1) rules0[0].Count = 0 // test prune useless rules rules1, err := NewRules(Voter, 4, `["-zone=sh", "+zone=bj"]`) - c.Assert(err, IsNil) - c.Assert(rules1, HasLen, 1) + require.NoError(t, err) + require.Len(t, rules1, 1) rules2, err := NewRules(Voter, 4, `["-zone=sh", "+zone=bj"]`) - c.Assert(err, IsNil) + require.NoError(t, err) bundle.Rules = append(bundle.Rules, rules0...) bundle.Rules = append(bundle.Rules, rules1...) bundle.Rules = append(bundle.Rules, rules2...) err = bundle.Tidy() - c.Assert(err, IsNil) - c.Assert(bundle.Rules, HasLen, 2) - c.Assert(bundle.Rules[0].ID, Equals, "1") - c.Assert(bundle.Rules[0].Constraints, HasLen, 3) - c.Assert(bundle.Rules[0].Constraints[2], DeepEquals, Constraint{ + require.NoError(t, err) + require.Len(t, bundle.Rules, 2) + require.Equal(t, "1", bundle.Rules[0].ID) + require.Len(t, bundle.Rules[0].Constraints, 3) + require.Equal(t, Constraint{ Op: NotIn, Key: EngineLabelKey, Values: []string{EngineLabelTiFlash}, - }) - c.Assert(bundle.Rules[1].ID, Equals, "2") + }, bundle.Rules[0].Constraints[2]) + require.Equal(t, "2", bundle.Rules[1].ID) // merge rules3, err := NewRules(Follower, 4, "") - c.Assert(err, IsNil) - c.Assert(rules3, HasLen, 1) + require.NoError(t, err) + require.Len(t, rules3, 1) rules4, err := NewRules(Follower, 5, "") - c.Assert(err, IsNil) - c.Assert(rules4, HasLen, 1) + require.NoError(t, err) + require.Len(t, rules4, 1) rules0[0].Role = Voter bundle.Rules = append(bundle.Rules, rules0...) @@ -881,19 +876,19 @@ func (s *testBundleSuite) TestTidy(c *C) { bundle.Rules = append(bundle.Rules, rules4...) chkfunc := func() { - c.Assert(err, IsNil) - c.Assert(bundle.Rules, HasLen, 3) - c.Assert(bundle.Rules[0].ID, Equals, "0") - c.Assert(bundle.Rules[1].ID, Equals, "1") - c.Assert(bundle.Rules[2].ID, Equals, "follower") - c.Assert(bundle.Rules[2].Count, Equals, 9) - c.Assert(bundle.Rules[2].Constraints, DeepEquals, Constraints{ + require.NoError(t, err) + require.Len(t, bundle.Rules, 3) + require.Equal(t, "0", bundle.Rules[0].ID) + require.Equal(t, "1", bundle.Rules[1].ID) + require.Equal(t, "follower", bundle.Rules[2].ID) + require.Equal(t, 9, bundle.Rules[2].Count) + require.Equal(t, Constraints{ { Op: NotIn, Key: EngineLabelKey, Values: []string{EngineLabelTiFlash}, }, - }) + }, bundle.Rules[2].Constraints) } err = bundle.Tidy() chkfunc() @@ -907,15 +902,13 @@ func (s *testBundleSuite) TestTidy(c *C) { // it should be stable bundle2 := bundle.Clone() err = bundle2.Tidy() - c.Assert(err, IsNil) - c.Assert(bundle2, DeepEquals, bundle) + require.NoError(t, err) + require.Equal(t, bundle, bundle2) bundle.Rules[2].Constraints = append(bundle.Rules[2].Constraints, Constraint{ Op: In, Key: EngineLabelKey, Values: []string{EngineLabelTiFlash}, }) - c.Log(bundle.Rules[2]) - err = bundle.Tidy() - c.Assert(errors.Is(err, ErrConflictingConstraints), IsTrue) + require.ErrorIs(t, bundle.Tidy(), ErrConflictingConstraints) } diff --git a/ddl/placement/common_test.go b/ddl/placement/common_test.go index 75c3285965dbd..b22226f29c948 100644 --- a/ddl/placement/common_test.go +++ b/ddl/placement/common_test.go @@ -17,19 +17,11 @@ package placement import ( "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testCommonSuite{}) - -type testCommonSuite struct{} - -func (t *testCommonSuite) TestGroup(c *C) { - c.Assert(GroupID(1), Equals, "TiDB_DDL_1") - c.Assert(GroupID(90), Equals, "TiDB_DDL_90") - c.Assert(GroupID(-1), Equals, "TiDB_DDL_-1") +func TestGroup(t *testing.T) { + require.Equal(t, "TiDB_DDL_1", GroupID(1)) + require.Equal(t, "TiDB_DDL_90", GroupID(90)) + require.Equal(t, "TiDB_DDL_-1", GroupID(-1)) } diff --git a/ddl/placement/constraint_test.go b/ddl/placement/constraint_test.go index a83aaa0098ab1..739d7bf6b5ba6 100644 --- a/ddl/placement/constraint_test.go +++ b/ddl/placement/constraint_test.go @@ -15,23 +15,20 @@ package placement import ( - "errors" + "fmt" + "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testConstraintSuite{}) - -type testConstraintSuite struct{} - -func (t *testConstraintSuite) TestNewFromYaml(c *C) { +func TestNewFromYaml(t *testing.T) { _, err := NewConstraintsFromYaml([]byte("[]")) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = NewConstraintsFromYaml([]byte("]")) - c.Assert(err, NotNil) + require.Error(t, err) } -func (t *testConstraintSuite) TestNew(c *C) { +func TestNewConstraint(t *testing.T) { type TestCase struct { name string input string @@ -114,19 +111,19 @@ func (t *testConstraintSuite) TestNew(c *C) { }, } - for _, t := range tests { - label, err := NewConstraint(t.input) - comment := Commentf("%s: %v", t.name, err) - if t.err == nil { - c.Assert(err, IsNil, comment) - c.Assert(label, DeepEquals, t.label, comment) + for _, test := range tests { + label, err := NewConstraint(test.input) + comment := fmt.Sprintf("%s: %v", test.name, err) + if test.err == nil { + require.NoError(t, err, comment) + require.Equal(t, test.label, label, comment) } else { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + require.ErrorIs(t, err, test.err, comment) } } } -func (t *testConstraintSuite) TestRestore(c *C) { +func TestRestoreConstraint(t *testing.T) { type TestCase struct { name string input Constraint @@ -136,7 +133,7 @@ func (t *testConstraintSuite) TestRestore(c *C) { var tests []TestCase input, err := NewConstraint("+zone=bj") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ name: "normal, op in", input: input, @@ -144,7 +141,7 @@ func (t *testConstraintSuite) TestRestore(c *C) { }) input, err = NewConstraint("+ zone = bj ") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ name: "normal with spaces, op in", input: input, @@ -152,7 +149,7 @@ func (t *testConstraintSuite) TestRestore(c *C) { }) input, err = NewConstraint("- zone = bj ") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ name: "normal with spaces, op not in", input: input, @@ -189,19 +186,19 @@ func (t *testConstraintSuite) TestRestore(c *C) { err: ErrInvalidConstraintFormat, }) - for _, t := range tests { - output, err := t.input.Restore() - comment := Commentf("%s: %v", t.name, err) - if t.err == nil { - c.Assert(err, IsNil, comment) - c.Assert(output, Equals, t.output, comment) + for _, test := range tests { + output, err := test.input.Restore() + comment := fmt.Sprintf("%s: %v", test.name, err) + if test.err == nil { + require.NoError(t, err, comment) + require.Equal(t, test.output, output, comment) } else { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + require.ErrorIs(t, err, test.err, comment) } } } -func (t *testConstraintSuite) TestCompatibleWith(c *C) { +func TestCompatibleWith(t *testing.T) { type TestCase struct { name string i1 Constraint @@ -211,9 +208,9 @@ func (t *testConstraintSuite) TestCompatibleWith(c *C) { var tests []TestCase i1, err := NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) i2, err := NewConstraint("-zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "case 2", i1, i2, @@ -221,9 +218,9 @@ func (t *testConstraintSuite) TestCompatibleWith(c *C) { }) i1, err = NewConstraint("+zone=bj") - c.Assert(err, IsNil) + require.NoError(t, err) i2, err = NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "case 3", i1, i2, @@ -231,9 +228,9 @@ func (t *testConstraintSuite) TestCompatibleWith(c *C) { }) i1, err = NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) i2, err = NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "case 1", i1, i2, @@ -241,9 +238,9 @@ func (t *testConstraintSuite) TestCompatibleWith(c *C) { }) i1, err = NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) i2, err = NewConstraint("+dc=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "normal 1", i1, i2, @@ -251,17 +248,16 @@ func (t *testConstraintSuite) TestCompatibleWith(c *C) { }) i1, err = NewConstraint("-zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) i2, err = NewConstraint("-zone=bj") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "normal 2", i1, i2, ConstraintCompatible, }) - for _, t := range tests { - comment := Commentf("%s", t.name) - c.Assert(t.i1.CompatibleWith(&t.i2), Equals, t.output, comment) + for _, test := range tests { + require.Equal(t, test.output, test.i1.CompatibleWith(&test.i2), test.name) } } diff --git a/ddl/placement/constraints_test.go b/ddl/placement/constraints_test.go index ef15c878943c9..17a4b03843255 100644 --- a/ddl/placement/constraints_test.go +++ b/ddl/placement/constraints_test.go @@ -15,30 +15,27 @@ package placement import ( - "errors" + "fmt" + "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testConstraintsSuite{}) - -type testConstraintsSuite struct{} - -func (t *testConstraintsSuite) TestNew(c *C) { +func TestNewConstraints(t *testing.T) { _, err := NewConstraints(nil) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = NewConstraints([]string{}) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = NewConstraints([]string{"+zonesh"}) - c.Assert(errors.Is(err, ErrInvalidConstraintFormat), IsTrue) + require.ErrorIs(t, err, ErrInvalidConstraintFormat) _, err = NewConstraints([]string{"+zone=sh", "-zone=sh"}) - c.Assert(errors.Is(err, ErrConflictingConstraints), IsTrue) + require.ErrorIs(t, err, ErrConflictingConstraints) } -func (t *testConstraintsSuite) TestAdd(c *C) { +func TestAdd(t *testing.T) { type TestCase struct { name string labels Constraints @@ -48,9 +45,9 @@ func (t *testConstraintsSuite) TestAdd(c *C) { var tests []TestCase labels, err := NewConstraints([]string{"+zone=sh"}) - c.Assert(err, IsNil) + require.NoError(t, err) label, err := NewConstraint("-zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "always false match", labels, label, @@ -58,9 +55,9 @@ func (t *testConstraintsSuite) TestAdd(c *C) { }) labels, err = NewConstraints([]string{"+zone=sh"}) - c.Assert(err, IsNil) + require.NoError(t, err) label, err = NewConstraint("+zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "duplicated constraints, skip", labels, label, @@ -78,7 +75,7 @@ func (t *testConstraintsSuite) TestAdd(c *C) { }) labels, err = NewConstraints([]string{"+zone=sh"}) - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "invalid label in operand", labels, Constraint{Op: "["}, @@ -98,28 +95,28 @@ func (t *testConstraintsSuite) TestAdd(c *C) { }) labels, err = NewConstraints([]string{"+zone=sh"}) - c.Assert(err, IsNil) + require.NoError(t, err) label, err = NewConstraint("-zone=bj") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "normal", labels, label, nil, }) - for _, t := range tests { - err := t.labels.Add(t.label) - comment := Commentf("%s: %v", t.name, err) - if t.err == nil { - c.Assert(err, IsNil, comment) - c.Assert(t.labels[len(t.labels)-1], DeepEquals, t.label, comment) + for _, test := range tests { + err := test.labels.Add(test.label) + comment := fmt.Sprintf("%s: %v", test.name, err) + if test.err == nil { + require.NoError(t, err, comment) + require.Equal(t, test.label, test.labels[len(test.labels)-1], comment) } else { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + require.ErrorIs(t, err, test.err, comment) } } } -func (t *testConstraintsSuite) TestRestore(c *C) { +func TestRestoreConstraints(t *testing.T) { type TestCase struct { name string input Constraints @@ -136,9 +133,9 @@ func (t *testConstraintsSuite) TestRestore(c *C) { }) input1, err := NewConstraint("+zone=bj") - c.Assert(err, IsNil) + require.NoError(t, err) input2, err := NewConstraint("-zone=sh") - c.Assert(err, IsNil) + require.NoError(t, err) tests = append(tests, TestCase{ "normal2", Constraints{input1, input2}, @@ -157,14 +154,14 @@ func (t *testConstraintsSuite) TestRestore(c *C) { ErrInvalidConstraintFormat, }) - for _, t := range tests { - res, err := t.input.Restore() - comment := Commentf("%s: %v", t.name, err) - if t.err == nil { - c.Assert(err, IsNil, comment) - c.Assert(res, Equals, t.output, comment) + for _, test := range tests { + res, err := test.input.Restore() + comment := fmt.Sprintf("%s: %v", test.name, err) + if test.err == nil { + require.NoError(t, err, comment) + require.Equal(t, test.output, res, comment) } else { - c.Assert(errors.Is(err, t.err), IsTrue, comment) + require.ErrorIs(t, err, test.err, comment) } } } diff --git a/ddl/placement/meta_bundle_test.go b/ddl/placement/meta_bundle_test.go index a556849121df8..f53599bd2e14c 100644 --- a/ddl/placement/meta_bundle_test.go +++ b/ddl/placement/meta_bundle_test.go @@ -19,8 +19,8 @@ import ( "encoding/hex" "encoding/json" "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" @@ -28,11 +28,10 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testMetaBundleSuite{}) - -type testMetaBundleSuite struct { +type metaBundleSuite struct { policy1 *model.PolicyInfo policy2 *model.PolicyInfo policy3 *model.PolicyInfo @@ -42,7 +41,8 @@ type testMetaBundleSuite struct { tbl4 *model.TableInfo } -func (s *testMetaBundleSuite) SetUpSuite(c *C) { +func createMetaBundleSuite() *metaBundleSuite { + s := new(metaBundleSuite) s.policy1 = &model.PolicyInfo{ ID: 11, Name: model.NewCIStr("p1"), @@ -124,154 +124,160 @@ func (s *testMetaBundleSuite) SetUpSuite(c *C) { ID: 104, Name: model.NewCIStr("t4"), } + return s } -func (s *testMetaBundleSuite) prepareMeta(c *C, store kv.Storage) { - c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - c.Assert(t.CreatePolicy(s.policy1), IsNil) - c.Assert(t.CreatePolicy(s.policy2), IsNil) - c.Assert(t.CreatePolicy(s.policy3), IsNil) +func (s *metaBundleSuite) prepareMeta(t *testing.T, store kv.Storage) { + err := kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + require.NoError(t, m.CreatePolicy(s.policy1)) + require.NoError(t, m.CreatePolicy(s.policy2)) + require.NoError(t, m.CreatePolicy(s.policy3)) return nil - }), IsNil) + }) + require.NoError(t, err) } -func (s *testMetaBundleSuite) TestNewTableBundle(c *C) { - store := newMockStore(c) - defer func() { - c.Assert(store.Close(), IsNil) - }() - s.prepareMeta(c, store) - c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) +func TestNewTableBundle(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { require.NoError(t, store.Close()) }() + + s := createMetaBundleSuite() + s.prepareMeta(t, store) + require.NoError(t, kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) // tbl1 - bundle, err := placement.NewTableBundle(t, s.tbl1) - c.Assert(err, IsNil) - s.checkTableBundle(c, s.tbl1, bundle) + bundle, err := placement.NewTableBundle(m, s.tbl1) + require.NoError(t, err) + s.checkTableBundle(t, s.tbl1, bundle) // tbl2 - bundle, err = placement.NewTableBundle(t, s.tbl2) - c.Assert(err, IsNil) - s.checkTableBundle(c, s.tbl2, bundle) + bundle, err = placement.NewTableBundle(m, s.tbl2) + require.NoError(t, err) + s.checkTableBundle(t, s.tbl2, bundle) // tbl3 - bundle, err = placement.NewTableBundle(t, s.tbl3) - c.Assert(err, IsNil) - s.checkTableBundle(c, s.tbl3, bundle) + bundle, err = placement.NewTableBundle(m, s.tbl3) + require.NoError(t, err) + s.checkTableBundle(t, s.tbl3, bundle) // tbl4 - bundle, err = placement.NewTableBundle(t, s.tbl4) - c.Assert(err, IsNil) - s.checkTableBundle(c, s.tbl4, bundle) + bundle, err = placement.NewTableBundle(m, s.tbl4) + require.NoError(t, err) + s.checkTableBundle(t, s.tbl4, bundle) return nil - }), IsNil) + })) } -func (s *testMetaBundleSuite) TestNewPartitionBundle(c *C) { - store := newMockStore(c) - defer func() { - c.Assert(store.Close(), IsNil) - }() - s.prepareMeta(c, store) +func TestNewPartitionBundle(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { require.NoError(t, store.Close()) }() - c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) + s := createMetaBundleSuite() + s.prepareMeta(t, store) + + require.NoError(t, kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) // tbl1.par0 - bundle, err := placement.NewPartitionBundle(t, s.tbl1.Partition.Definitions[0]) - c.Assert(err, IsNil) - s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[0], bundle) + bundle, err := placement.NewPartitionBundle(m, s.tbl1.Partition.Definitions[0]) + require.NoError(t, err) + s.checkPartitionBundle(t, s.tbl1.Partition.Definitions[0], bundle) // tbl1.par1 - bundle, err = placement.NewPartitionBundle(t, s.tbl1.Partition.Definitions[1]) - c.Assert(err, IsNil) - s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[1], bundle) + bundle, err = placement.NewPartitionBundle(m, s.tbl1.Partition.Definitions[1]) + require.NoError(t, err) + s.checkPartitionBundle(t, s.tbl1.Partition.Definitions[1], bundle) return nil - }), IsNil) + })) } -func (s *testMetaBundleSuite) TestNewPartitionListBundles(c *C) { - store := newMockStore(c) - defer func() { - c.Assert(store.Close(), IsNil) - }() - s.prepareMeta(c, store) +func TestNewPartitionListBundles(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { require.NoError(t, store.Close()) }() + + s := createMetaBundleSuite() + s.prepareMeta(t, store) - c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) + require.NoError(t, kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) - bundles, err := placement.NewPartitionListBundles(t, s.tbl1.Partition.Definitions) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 1) - s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[1], bundles[0]) + bundles, err := placement.NewPartitionListBundles(m, s.tbl1.Partition.Definitions) + require.NoError(t, err) + require.Len(t, bundles, 1) + s.checkPartitionBundle(t, s.tbl1.Partition.Definitions[1], bundles[0]) - bundles, err = placement.NewPartitionListBundles(t, []model.PartitionDefinition{}) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 0) + bundles, err = placement.NewPartitionListBundles(m, []model.PartitionDefinition{}) + require.NoError(t, err) + require.Len(t, bundles, 0) - bundles, err = placement.NewPartitionListBundles(t, []model.PartitionDefinition{ + bundles, err = placement.NewPartitionListBundles(m, []model.PartitionDefinition{ s.tbl1.Partition.Definitions[0], s.tbl1.Partition.Definitions[2], }) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 0) + require.NoError(t, err) + require.Len(t, bundles, 0) return nil - }), IsNil) + })) } -func (s *testMetaBundleSuite) TestNewFullTableBundles(c *C) { - store := newMockStore(c) - defer func() { - c.Assert(store.Close(), IsNil) - }() - s.prepareMeta(c, store) +func TestNewFullTableBundles(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { require.NoError(t, store.Close()) }() + + s := createMetaBundleSuite() + s.prepareMeta(t, store) - c.Assert(kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) + require.NoError(t, kv.RunInNewTxn(context.TODO(), store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) - bundles, err := placement.NewFullTableBundles(t, s.tbl1) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 2) - s.checkTableBundle(c, s.tbl1, bundles[0]) - s.checkPartitionBundle(c, s.tbl1.Partition.Definitions[1], bundles[1]) + bundles, err := placement.NewFullTableBundles(m, s.tbl1) + require.NoError(t, err) + require.Len(t, bundles, 2) + s.checkTableBundle(t, s.tbl1, bundles[0]) + s.checkPartitionBundle(t, s.tbl1.Partition.Definitions[1], bundles[1]) - bundles, err = placement.NewFullTableBundles(t, s.tbl2) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 1) - s.checkPartitionBundle(c, s.tbl2.Partition.Definitions[0], bundles[0]) + bundles, err = placement.NewFullTableBundles(m, s.tbl2) + require.NoError(t, err) + require.Len(t, bundles, 1) + s.checkPartitionBundle(t, s.tbl2.Partition.Definitions[0], bundles[0]) - bundles, err = placement.NewFullTableBundles(t, s.tbl3) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 1) - s.checkTableBundle(c, s.tbl3, bundles[0]) + bundles, err = placement.NewFullTableBundles(m, s.tbl3) + require.NoError(t, err) + require.Len(t, bundles, 1) + s.checkTableBundle(t, s.tbl3, bundles[0]) - bundles, err = placement.NewFullTableBundles(t, s.tbl4) - c.Assert(err, IsNil) - c.Assert(len(bundles), Equals, 0) + bundles, err = placement.NewFullTableBundles(m, s.tbl4) + require.NoError(t, err) + require.Len(t, bundles, 0) return nil - }), IsNil) + })) } -func (s *testMetaBundleSuite) checkTwoJSONObjectEquals(c *C, expected interface{}, got interface{}) { +func (s *metaBundleSuite) checkTwoJSONObjectEquals(t *testing.T, expected interface{}, got interface{}) { expectedJSON, err := json.Marshal(expected) - c.Assert(err, IsNil) + require.NoError(t, err) expectedStr := string(expectedJSON) gotJSON, err := json.Marshal(got) - c.Assert(err, IsNil) + require.NoError(t, err) gotStr := string(gotJSON) - c.Assert(gotStr, Equals, expectedStr) + require.Equal(t, expectedStr, gotStr) } -func (s *testMetaBundleSuite) checkTableBundle(c *C, tbl *model.TableInfo, got *placement.Bundle) { +func (s *metaBundleSuite) checkTableBundle(t *testing.T, tbl *model.TableInfo, got *placement.Bundle) { if tbl.PlacementPolicyRef == nil { - c.Assert(got, IsNil) + require.Nil(t, got) return } @@ -279,7 +285,7 @@ func (s *testMetaBundleSuite) checkTableBundle(c *C, tbl *model.TableInfo, got * ID: fmt.Sprintf("TiDB_DDL_%d", tbl.ID), Index: placement.RuleIndexTable, Override: true, - Rules: s.expectedRules(c, tbl.PlacementPolicyRef), + Rules: s.expectedRules(t, tbl.PlacementPolicyRef), } for idx, rule := range expected.Rules { @@ -292,7 +298,7 @@ func (s *testMetaBundleSuite) checkTableBundle(c *C, tbl *model.TableInfo, got * if tbl.Partition != nil { for _, par := range tbl.Partition.Definitions { - rules := s.expectedRules(c, tbl.PlacementPolicyRef) + rules := s.expectedRules(t, tbl.PlacementPolicyRef) for idx, rule := range rules { rule.GroupID = expected.ID rule.Index = placement.RuleIndexPartition @@ -304,12 +310,12 @@ func (s *testMetaBundleSuite) checkTableBundle(c *C, tbl *model.TableInfo, got * } } - s.checkTwoJSONObjectEquals(c, expected, got) + s.checkTwoJSONObjectEquals(t, expected, got) } -func (s *testMetaBundleSuite) checkPartitionBundle(c *C, def model.PartitionDefinition, got *placement.Bundle) { +func (s *metaBundleSuite) checkPartitionBundle(t *testing.T, def model.PartitionDefinition, got *placement.Bundle) { if def.PlacementPolicyRef == nil { - c.Assert(got, IsNil) + require.Nil(t, got) return } @@ -317,7 +323,7 @@ func (s *testMetaBundleSuite) checkPartitionBundle(c *C, def model.PartitionDefi ID: fmt.Sprintf("TiDB_DDL_%d", def.ID), Index: placement.RuleIndexPartition, Override: true, - Rules: s.expectedRules(c, def.PlacementPolicyRef), + Rules: s.expectedRules(t, def.PlacementPolicyRef), } for idx, rule := range expected.Rules { @@ -328,10 +334,10 @@ func (s *testMetaBundleSuite) checkPartitionBundle(c *C, def model.PartitionDefi rule.EndKeyHex = hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(def.ID+1))) } - s.checkTwoJSONObjectEquals(c, expected, got) + s.checkTwoJSONObjectEquals(t, expected, got) } -func (s *testMetaBundleSuite) expectedRules(c *C, ref *model.PolicyRefInfo) []*placement.Rule { +func (s *metaBundleSuite) expectedRules(t *testing.T, ref *model.PolicyRefInfo) []*placement.Rule { if ref == nil { return []*placement.Rule{} } @@ -345,18 +351,14 @@ func (s *testMetaBundleSuite) expectedRules(c *C, ref *model.PolicyRefInfo) []*p case s.policy3.ID: policy = s.policy3 default: - c.FailNow() + t.FailNow() } - c.Assert(ref.Name, Equals, policy.Name) + + require.Equal(t, policy.Name, ref.Name) settings := policy.PlacementSettings bundle, err := placement.NewBundleFromOptions(settings) - c.Assert(err, IsNil) - return bundle.Rules -} + require.NoError(t, err) -func newMockStore(c *C) kv.Storage { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - return store + return bundle.Rules } diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go index a48dc38faa33c..bee9807b486bd 100644 --- a/ddl/placement/rule_test.go +++ b/ddl/placement/rule_test.go @@ -16,10 +16,10 @@ package placement import ( "errors" + "fmt" "reflect" "testing" - . "github.com/pingcap/check" "github.com/stretchr/testify/require" ) @@ -32,24 +32,7 @@ func TestClone(t *testing.T) { require.Equal(t, &Rule{ID: "121"}, newRule) } -func matchRules(t1, t2 []*Rule, prefix string, c *C) { - c.Assert(len(t2), Equals, len(t1), Commentf(prefix)) - for i := range t1 { - found := false - for j := range t2 { - ok, _ := DeepEquals.Check([]interface{}{t2[j], t1[i]}, []string{}) - if ok { - found = true - break - } - } - if !found { - c.Errorf("%s\n\ncan not found %d rule\n%+v\n%+v", prefix, i, t1[i], t2) - } - } -} - -func matchRulesT(t1, t2 []*Rule, prefix string, t *testing.T) { +func matchRules(t1, t2 []*Rule, prefix string, t *testing.T) { require.Equal(t, len(t2), len(t1), prefix) for i := range t1 { found := false @@ -157,11 +140,11 @@ func TestNewRuleAndNewRules(t *testing.T) { }) for _, tt := range tests { - comment := Commentf("[%s]", tt.name) + comment := fmt.Sprintf("[%s]", tt.name) output, err := NewRules(Voter, tt.replicas, tt.input) if tt.err == nil { require.NoError(t, err, comment) - matchRulesT(tt.output, output, comment.CheckCommentString(), t) + matchRules(tt.output, output, comment, t) } else { require.True(t, errors.Is(err, tt.err), "[%s]\n%s\n%s\n", tt.name, err, tt.err) } diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go index 7cfcbd0a5d9f9..44d40049e5a05 100644 --- a/ddl/placement_policy_test.go +++ b/ddl/placement_policy_test.go @@ -399,7 +399,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { // test for normal cases tk.MustExec("alter placement policy x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj,sh\" NULL")) - tk.MustQuery("select * from information_schema.placement_policies where policy_name = 'x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj,sh 0 0")) + tk.MustQuery("select * from information_schema.placement_policies where policy_name = 'x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj,sh 2 0")) checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter placement policy x " + @@ -407,7 +407,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { "REGIONS=\"bj\" " + "SCHEDULE=\"EVEN\"") tk.MustQuery("show placement where target='POLICY x'").Check(testkit.Rows("POLICY x PRIMARY_REGION=\"bj\" REGIONS=\"bj\" SCHEDULE=\"EVEN\" NULL")) - tk.MustQuery("select * from INFORMATION_SCHEMA.PLACEMENT_POLICIES WHERE POLICY_NAME='x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj EVEN 0 0")) + tk.MustQuery("select * from INFORMATION_SCHEMA.PLACEMENT_POLICIES WHERE POLICY_NAME='x'").Check(testkit.Rows(strconv.FormatInt(policy.ID, 10) + " def x bj bj EVEN 2 0")) checkExistTableBundlesInPD(c, s.dom, "test", "tp") tk.MustExec("alter placement policy x " + @@ -435,7 +435,7 @@ func (s *testDBSuite6) TestAlterPlacementPolicy(c *C) { "CATALOG_NAME,POLICY_NAME," + "PRIMARY_REGION,REGIONS,CONSTRAINTS,LEADER_CONSTRAINTS,FOLLOWER_CONSTRAINTS,LEARNER_CONSTRAINTS," + "SCHEDULE,FOLLOWERS,LEARNERS FROM INFORMATION_SCHEMA.placement_policies WHERE POLICY_NAME='x'").Check( - testkit.Rows("def x [+disk=ssd] [+region=sh] 0 3"), + testkit.Rows("def x [+disk=ssd] [+region=sh] 2 3"), ) checkExistTableBundlesInPD(c, s.dom, "test", "tp") diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index 8c586a872651f..ade5f33fdf2a7 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -17,21 +17,24 @@ package ddl_test import ( "fmt" "sort" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/domain" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) // TODO: Remove in https://github.com/pingcap/tidb/issues/27971 or change to use SQL PLACEMENT POLICY -func (s *testDBSuite1) TestPlacementPolicyCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPlacementPolicyCache(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_exchange_partition = 1") defer func() { @@ -46,10 +49,10 @@ func (s *testDBSuite1) TestPlacementPolicyCache(c *C) { tk.MustExec(`create table t1(id int) partition by range(id) (partition p0 values less than (100), partition p1 values less than (200))`) - is := s.dom.InfoSchema() + is := dom.InfoSchema() tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) partDefs := tb.Meta().GetPartitionInfo().Definitions sort.Slice(partDefs, func(i, j int) bool { return partDefs[i].Name.L < partDefs[j].Name.L }) @@ -96,8 +99,10 @@ func (s *testDBSuite1) TestPlacementPolicyCache(c *C) { //tk.MustQuery("select * from information_schema.placement_policy").Check(testkit.Rows()) } -func (s *testSerialDBSuite) TestTxnScopeConstraint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTxnScopeConstraint(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") defer func() { @@ -112,10 +117,10 @@ PARTITION BY RANGE (c) ( PARTITION p3 VALUES LESS THAN (21) );`) - is := s.dom.InfoSchema() + is := dom.InfoSchema() tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) partDefs := tb.Meta().GetPartitionInfo().Definitions for _, def := range partDefs { @@ -220,12 +225,8 @@ PARTITION BY RANGE (c) ( } for _, testcase := range testCases { - c.Log(testcase.name) failpoint.Enable("tikvclient/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) - se, err := session.CreateSession4Test(s.store) - c.Check(err, IsNil) - tk.Se = se tk.MustExec("use test") tk.MustExec("set global tidb_enable_local_txn = on;") tk.MustExec(fmt.Sprintf("set @@txn_scope = %v", testcase.txnScope)) @@ -238,18 +239,20 @@ PARTITION BY RANGE (c) ( _, err = tk.Exec(testcase.sql) } if testcase.err == nil { - c.Assert(err, IsNil) + require.NoError(t, err) } else { - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, testcase.err.Error()) + require.Error(t, err) + require.Regexp(t, testcase.err.Error(), err.Error()) } tk.MustExec("set global tidb_enable_local_txn = off;") failpoint.Disable("tikvclient/injectTxnScope") } } -func (s *testDBSuite6) TestCreateSchemaWithPlacement(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateSchemaWithPlacement(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop schema if exists SchemaPolicyPlacementTest") defer func() { tk.MustExec("drop schema if exists SchemaPolicyPlacementTest") @@ -282,15 +285,17 @@ func (s *testDBSuite6) TestCreateSchemaWithPlacement(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`PolicyTableTest` */")) tk.MustQuery("SELECT TABLE_CATALOG, TABLE_SCHEMA, TABLE_NAME, TIDB_PLACEMENT_POLICY_NAME FROM information_schema.Tables WHERE TABLE_SCHEMA='SchemaPolicyPlacementTest' AND TABLE_NAME = 'UsePolicy'").Check(testkit.Rows(`def SchemaPolicyPlacementTest UsePolicy PolicyTableTest`)) - is := s.dom.InfoSchema() + is := dom.InfoSchema() db, ok := is.SchemaByName(model.NewCIStr("SchemaPolicyPlacementTest")) - c.Assert(ok, IsTrue) - c.Assert(db.PlacementPolicyRef, NotNil) - c.Assert(db.PlacementPolicyRef.Name.O, Equals, "PolicySchemaTest") + require.True(t, ok) + require.NotNil(t, db.PlacementPolicyRef) + require.Equal(t, "PolicySchemaTest", db.PlacementPolicyRef.Name.O) } -func (s *testDBSuite6) TestAlterDBPlacement(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterDBPlacement(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists TestAlterDB;") tk.MustExec("create database TestAlterDB;") tk.MustExec("use TestAlterDB") @@ -374,8 +379,10 @@ func (s *testDBSuite6) TestAlterDBPlacement(c *C) { )) } -func (s *testDBSuite6) TestPlacementMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPlacementMode(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop database if exists db1") tk.MustExec("drop database if exists db2") @@ -406,7 +413,7 @@ func (s *testDBSuite6) TestPlacementMode(c *C) { // invalid values err := tk.ExecToErr("set tidb_placement_mode='aaa'") - c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_placement_mode' can't be set to the value of 'aaa'") + require.EqualError(t, err, "[variable:1231]Variable 'tidb_placement_mode' can't be set to the value of 'aaa'") // ignore mode tk.MustExec("set tidb_placement_mode='ignore'") @@ -596,58 +603,60 @@ func (s *testDBSuite6) TestPlacementMode(c *C) { // create tableWithInfo in ignore mode tk.MustExec("drop table if exists t2") - tbl, err := s.getClonedTable("test", "t1") - c.Assert(err, IsNil) - c.Assert(tbl.PlacementPolicyRef, NotNil) + tbl, err := getClonedTableFromDomain("test", "t1", dom) + require.NoError(t, err) + require.NotNil(t, tbl.PlacementPolicyRef) tbl.Name = model.NewCIStr("t2") - err = s.dom.DDL().CreateTableWithInfo(tk.Se, model.NewCIStr("test"), tbl, ddl.OnExistError) - c.Assert(err, IsNil) + err = dom.DDL().CreateTableWithInfo(tk.Session(), model.NewCIStr("test"), tbl, ddl.OnExistError) + require.NoError(t, err) tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + " `id` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='aaa'")) // createTableWithInfo in ignore mode (policy not exists) tk.MustExec("drop table if exists t2") - tbl, err = s.getClonedTable("test", "t1") - c.Assert(err, IsNil) - c.Assert(tbl.PlacementPolicyRef, NotNil) + tbl, err = getClonedTableFromDomain("test", "t1", dom) + require.NoError(t, err) + require.NotNil(t, tbl.PlacementPolicyRef) tbl.Name = model.NewCIStr("t2") tbl.PlacementPolicyRef.Name = model.NewCIStr("pxx") - err = s.dom.DDL().CreateTableWithInfo(tk.Se, model.NewCIStr("test"), tbl, ddl.OnExistError) - c.Assert(err, IsNil) + err = dom.DDL().CreateTableWithInfo(tk.Session(), model.NewCIStr("test"), tbl, ddl.OnExistError) + require.NoError(t, err) tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + " `id` int(11) DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin COMMENT='aaa'")) // createSchemaWithInfo in ignore mode tk.MustExec("drop database if exists db2") - db1, ok := s.getClonedDatabase("db1") - c.Assert(ok, IsTrue) - c.Assert(db1.PlacementPolicyRef, NotNil) + db1, ok := getClonedDatabaseFromDomain("db1", dom) + require.True(t, ok) + require.NotNil(t, db1.PlacementPolicyRef) db1.Name = model.NewCIStr("db2") - err = s.dom.DDL().CreateSchemaWithInfo(tk.Se, db1, ddl.OnExistError) - c.Assert(err, IsNil) + err = dom.DDL().CreateSchemaWithInfo(tk.Session(), db1, ddl.OnExistError) + require.NoError(t, err) tk.MustQuery("show create database db2").Check(testkit.Rows("db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */")) // createSchemaWithInfo in ignore mode (policy not exists) tk.MustExec("drop database if exists db2") - db1, ok = s.getClonedDatabase("db1") - c.Assert(ok, IsTrue) - c.Assert(db1.PlacementPolicyRef, NotNil) + db1, ok = getClonedDatabaseFromDomain("db1", dom) + require.True(t, ok) + require.NotNil(t, db1.PlacementPolicyRef) db1.Name = model.NewCIStr("db2") db1.PlacementPolicyRef.Name = model.NewCIStr("pxx") - err = s.dom.DDL().CreateSchemaWithInfo(tk.Se, db1, ddl.OnExistError) - c.Assert(err, IsNil) + err = dom.DDL().CreateSchemaWithInfo(tk.Session(), db1, ddl.OnExistError) + require.NoError(t, err) tk.MustQuery("show create database db2").Check(testkit.Rows("db2 CREATE DATABASE `db2` /*!40100 DEFAULT CHARACTER SET utf8mb4 */")) } -func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { - tk := testkit.NewTestKit(c, s.store) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) +func TestPlacementTiflashCheck(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`)) defer func() { err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") - c.Assert(err, IsNil) + require.NoError(t, err) }() tk.MustExec("use test") @@ -665,9 +674,9 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { tk.MustExec("alter table tp set tiflash replica 1") err := tk.ExecToErr("alter table tp placement policy p1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) err = tk.ExecToErr("alter table tp partition p0 placement policy p1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -678,11 +687,11 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { tk.MustExec("drop table tp") tk.MustExec(`CREATE TABLE tp (id INT) placement policy p1 PARTITION BY RANGE (id) ( - PARTITION p0 VALUES LESS THAN (100), - PARTITION p1 VALUES LESS THAN (1000) + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (1000) )`) err = tk.ExecToErr("alter table tp set tiflash replica 1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -693,11 +702,11 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { tk.MustExec("drop table tp") tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) ( - PARTITION p0 VALUES LESS THAN (100) placement policy p1 , - PARTITION p1 VALUES LESS THAN (1000) + PARTITION p0 VALUES LESS THAN (100) placement policy p1 , + PARTITION p1 VALUES LESS THAN (1000) )`) err = tk.ExecToErr("alter table tp set tiflash replica 1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -708,11 +717,11 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { tk.MustExec("drop table tp") tk.MustExec(`CREATE TABLE tp (id INT) PLACEMENT POLICY p1 PARTITION BY RANGE (id) ( - PARTITION p0 VALUES LESS THAN (100), - PARTITION p1 VALUES LESS THAN (1000) + PARTITION p0 VALUES LESS THAN (100), + PARTITION p1 VALUES LESS THAN (1000) )`) err = tk.ExecToErr("alter table tp set tiflash replica 1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -723,11 +732,11 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { tk.MustExec("drop table tp") tk.MustExec(`CREATE TABLE tp (id INT) PARTITION BY RANGE (id) ( - PARTITION p0 VALUES LESS THAN (100) PLACEMENT POLICY p1, - PARTITION p1 VALUES LESS THAN (1000) + PARTITION p0 VALUES LESS THAN (100) PLACEMENT POLICY p1, + PARTITION p1 VALUES LESS THAN (1000) )`) err = tk.ExecToErr("alter table tp set tiflash replica 1") - c.Assert(ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err), IsTrue) + require.True(t, ddl.ErrIncompatibleTiFlashAndPlacement.Equal(err)) tk.MustQuery("show create table tp").Check(testkit.Rows("" + "tp CREATE TABLE `tp` (\n" + " `id` int(11) DEFAULT NULL\n" + @@ -736,3 +745,28 @@ func (s *testDBSuite6) TestPlacementTiflashCheck(c *C) { "(PARTITION `p0` VALUES LESS THAN (100) /*T![placement] PLACEMENT POLICY=`p1` */,\n" + " PARTITION `p1` VALUES LESS THAN (1000))")) } + +func getClonedTableFromDomain(dbName string, tableName string, dom *domain.Domain) (*model.TableInfo, error) { + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(dbName), model.NewCIStr(tableName)) + if err != nil { + return nil, err + } + + tblMeta := tbl.Meta() + tblMeta = tblMeta.Clone() + policyRef := *tblMeta.PlacementPolicyRef + tblMeta.PlacementPolicyRef = &policyRef + return tblMeta, nil +} + +func getClonedDatabaseFromDomain(dbName string, dom *domain.Domain) (*model.DBInfo, bool) { + db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(dbName)) + if !ok { + return nil, ok + } + + db = db.Clone() + policyRef := *db.PlacementPolicyRef + db.PlacementPolicyRef = &policyRef + return db, true +} diff --git a/ddl/reorg.go b/ddl/reorg.go index deeac2bfdb8ae..5fbe012bff18e 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -89,7 +89,10 @@ func newContext(store kv.Storage) sessionctx.Context { c := mock.NewContext() c.Store = store c.GetSessionVars().SetStatusFlag(mysql.ServerStatusAutocommit, false) - c.GetSessionVars().StmtCtx.TimeZone = time.UTC + + tz := *time.UTC + c.GetSessionVars().TimeZone = &tz + c.GetSessionVars().StmtCtx.TimeZone = &tz return c } @@ -199,6 +202,7 @@ func (w *worker) runReorgJob(t *meta.Meta, reorgInfo *reorgInfo, tblInfo *model. SQLMode: mysql.ModeNone, Warnings: make(map[errors.ErrorID]*terror.Error), WarningsCount: make(map[errors.ErrorID]int64), + Location: &model.TimeZoneLocation{Name: time.UTC.String(), Offset: 0}, } } if w.reorgCtx.doneCh == nil { @@ -340,11 +344,7 @@ func getTableTotalCount(w *worker, tblInfo *model.TableInfo) int64 { return statistics.PseudoRowCount } sql := "select table_rows from information_schema.tables where tidb_table_id=%?;" - stmt, err := executor.ParseWithParams(w.ddlJobCtx, true, sql, tblInfo.ID) - if err != nil { - return statistics.PseudoRowCount - } - rows, _, err := executor.ExecRestrictedStmt(w.ddlJobCtx, stmt) + rows, _, err := executor.ExecRestrictedSQL(w.ddlJobCtx, nil, sql, tblInfo.ID) if err != nil { return statistics.PseudoRowCount } diff --git a/ddl/restart_test.go b/ddl/restart_test.go index 1aa962bbbfa7f..0344b88caae37 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -92,17 +92,16 @@ func testRunInterruptedJob(t *testing.T, d *ddl, job *model.Job) { ticker := time.NewTicker(d.lease * 1) defer ticker.Stop() -LOOP: for { select { case <-ticker.C: err := d.Stop() - require.Nil(t, err) + require.NoError(t, err) d.restartWorkers(context.Background()) time.Sleep(time.Millisecond * 20) case err := <-done: require.Nil(t, err) - break LOOP + return } } } diff --git a/ddl/schema_test.go b/ddl/schema_test.go index 9594331b77298..5034a32dc155d 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -141,7 +141,7 @@ func testCheckSchemaState(test *testing.T, d *ddl, dbInfo *model.DBInfo, state m } } -func TestSchema(t *testing.T) { +func ExportTestSchema(t *testing.T) { store := testCreateStore(t, "test_schema") defer func() { err := store.Close() diff --git a/ddl/sequence_test.go b/ddl/sequence_test.go index 38fe93941523f..d5895e2e83f24 100644 --- a/ddl/sequence_test.go +++ b/ddl/sequence_test.go @@ -17,8 +17,8 @@ package ddl_test import ( "strconv" "testing" + "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl" mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/auth" @@ -26,16 +26,15 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/table/tables" - testkit2 "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testSequenceSuite{&testDBSuite{}}) - -type testSequenceSuite struct{ *testDBSuite } - -func (s *testSequenceSuite) TestCreateSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") tk.MustGetErrCode("create sequence `seq `", mysql.ErrWrongTableName) @@ -64,40 +63,44 @@ func (s *testSequenceSuite) TestCreateSequence(c *C) { tk.MustGetErrCode("create sequence seq CHARSET=utf8", mysql.ErrSequenceUnsupportedTableOption) _, err := tk.Exec("create sequence seq comment=\"test\"") - c.Assert(err, IsNil) - - sequenceTable := testGetTableByName(c, s.s, "test", "seq") - c.Assert(sequenceTable.Meta().IsSequence(), Equals, true) - c.Assert(sequenceTable.Meta().Sequence.Increment, Equals, model.DefaultSequenceIncrementValue) - c.Assert(sequenceTable.Meta().Sequence.Start, Equals, model.DefaultPositiveSequenceStartValue) - c.Assert(sequenceTable.Meta().Sequence.MinValue, Equals, model.DefaultPositiveSequenceMinValue) - c.Assert(sequenceTable.Meta().Sequence.MaxValue, Equals, model.DefaultPositiveSequenceMaxValue) - c.Assert(sequenceTable.Meta().Sequence.Cache, Equals, true) - c.Assert(sequenceTable.Meta().Sequence.CacheValue, Equals, model.DefaultSequenceCacheValue) - c.Assert(sequenceTable.Meta().Sequence.Cycle, Equals, false) + require.NoError(t, err) + + sequenceTable := testkit.TestGetTableByName(t, tk.Session(), "test", "seq") + + require.Equal(t, true, sequenceTable.Meta().IsSequence()) + require.Equal(t, model.DefaultSequenceIncrementValue, sequenceTable.Meta().Sequence.Increment) + require.Equal(t, model.DefaultPositiveSequenceStartValue, sequenceTable.Meta().Sequence.Start) + require.Equal(t, model.DefaultPositiveSequenceMinValue, sequenceTable.Meta().Sequence.MinValue) + require.Equal(t, model.DefaultPositiveSequenceMaxValue, sequenceTable.Meta().Sequence.MaxValue) + require.Equal(t, true, sequenceTable.Meta().Sequence.Cache) + require.Equal(t, model.DefaultSequenceCacheValue, sequenceTable.Meta().Sequence.CacheValue) + require.Equal(t, false, sequenceTable.Meta().Sequence.Cycle) // Test create privilege. tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil)) + tk1.SetSession(se) // grant the myuser the access to database test. tk.MustExec("grant select on test.* to 'myuser'@'localhost'") tk1.MustExec("use test") _, err = tk1.Exec("create sequence my_seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1142]CREATE command denied to user 'myuser'@'localhost' for table 'my_seq'") + require.Error(t, err) + require.EqualError(t, err, "[planner:1142]CREATE command denied to user 'myuser'@'localhost' for table 'my_seq'") } // Test for sequence still works with a infoschema attached by temporary table -func (s *testSequenceSuite) TestIssue28881(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue28881(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists s") tk.MustExec("create sequence s") @@ -108,8 +111,11 @@ func (s *testSequenceSuite) TestIssue28881(c *C) { tk.MustQuery("select lastval(s)").Check(testkit.Rows("1")) } -func (s *testSequenceSuite) TestDropSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDropSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") @@ -119,49 +125,49 @@ func (s *testSequenceSuite) TestDropSequence(c *C) { // Test non-existed sequence can't drop successfully. tk.MustExec("create sequence seq") _, err := tk.Exec("drop sequence seq, seq2") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:4139]Unknown SEQUENCE: 'test.seq2'") + require.Error(t, err) + require.EqualError(t, err, "[schema:4139]Unknown SEQUENCE: 'test.seq2'") // Test the specified object is not sequence. tk.MustExec("create table seq3 (a int)") _, err = tk.Exec("drop sequence seq3") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongObject), IsTrue) + require.Error(t, err) + require.True(t, terror.ErrorEqual(err, ddl.ErrWrongObject)) // Test schema is not exist. _, err = tk.Exec("drop sequence unknown.seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:4139]Unknown SEQUENCE: 'unknown.seq'") + require.Error(t, err) + require.EqualError(t, err, "[schema:4139]Unknown SEQUENCE: 'unknown.seq'") // Test drop sequence successfully. tk.MustExec("create sequence seq") _, err = tk.Exec("drop sequence seq") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("drop sequence seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:4139]Unknown SEQUENCE: 'test.seq'") + require.Error(t, err) + require.EqualError(t, err, "[schema:4139]Unknown SEQUENCE: 'test.seq'") // Test drop table when the object is a sequence. tk.MustExec("create sequence seq") _, err = tk.Exec("drop table seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.seq'") + require.Error(t, err) + require.EqualError(t, err, "[schema:1051]Unknown table 'test.seq'") // Test drop view when the object is a sequence. _, err = tk.Exec("drop view seq") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, ddl.ErrWrongObject), IsTrue) + require.Error(t, err) + require.True(t, terror.ErrorEqual(err, ddl.ErrWrongObject)) tk.MustExec("drop sequence seq") // Test drop privilege. tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil)) + tk1.SetSession(se) // grant the myuser the access to database test. tk.MustExec("create sequence my_seq") @@ -169,16 +175,19 @@ func (s *testSequenceSuite) TestDropSequence(c *C) { tk1.MustExec("use test") _, err = tk1.Exec("drop sequence my_seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1142]DROP command denied to user 'myuser'@'localhost' for table 'my_seq'") + require.Error(t, err) + require.EqualError(t, err, "[planner:1142]DROP command denied to user 'myuser'@'localhost' for table 'my_seq'") // Test for `drop sequence if exists`. tk.MustExec("drop sequence if exists seq_if_exists") tk.MustQuery("show warnings;").Check(testkit.Rows("Note 4139 Unknown SEQUENCE: 'test.seq_if_exists'")) } -func (s *testSequenceSuite) TestShowCreateSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("drop sequence if exists seq") @@ -189,11 +198,11 @@ func (s *testSequenceSuite) TestShowCreateSequence(c *C) { tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil)) + tk1.SetSession(se) // Grant the myuser the access to table t in database test, but sequence seq. tk.MustExec("grant select on test.t to 'myuser'@'localhost'") @@ -201,8 +210,8 @@ func (s *testSequenceSuite) TestShowCreateSequence(c *C) { tk1.MustExec("use test") tk1.MustExec("show create table t") _, err = tk1.Exec("show create sequence seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1142]SHOW command denied to user 'myuser'@'localhost' for table 'seq'") + require.Error(t, err) + require.EqualError(t, err, "[planner:1142]SHOW command denied to user 'myuser'@'localhost' for table 'seq'") // Grant the myuser the access to sequence seq in database test. tk.MustExec("grant select on test.seq to 'myuser'@'localhost'") @@ -246,8 +255,8 @@ func (s *testSequenceSuite) TestShowCreateSequence(c *C) { tk.MustExec("drop sequence if exists seq") tk.MustExec("create table seq (a int)") err = tk.QueryToErr("show create sequence seq") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:1347]'test.seq' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[executor:1347]'test.seq' is not SEQUENCE") tk.MustExec("drop table if exists seq") // Test use the show create sequence result to create sequence. @@ -258,8 +267,11 @@ func (s *testSequenceSuite) TestShowCreateSequence(c *C) { tk.MustExec(showString) } -func (s *testSequenceSuite) TestSequenceAsDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceAsDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") tk.MustExec("create sequence seq") @@ -297,8 +309,11 @@ func (s *testSequenceSuite) TestSequenceAsDefaultValue(c *C) { tk.MustExec("alter table t5 change column c c int default next value for seq") } -func (s *testSequenceSuite) TestSequenceFunction(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") tk.MustExec("drop sequence if exists seq1") @@ -374,7 +389,7 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("8")) err := tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") tk.MustExec("drop sequence if exists seq") tk.MustExec("create sequence seq increment = 3 start = 3 maxvalue = 9 nocycle") @@ -382,7 +397,7 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select nextval(seq)").Check(testkit.Rows("6")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("9")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") // test negative-growth sequence tk.MustExec("drop sequence if exists seq") @@ -410,14 +425,14 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-2")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-6")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") tk.MustExec("drop sequence if exists seq") tk.MustExec("create sequence seq increment = -3 start = 2 minvalue -2 maxvalue 10") tk.MustQuery("select nextval(seq)").Check(testkit.Rows("2")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-1")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") // test sequence setval function. tk.MustExec("drop sequence if exists seq") @@ -443,14 +458,14 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select setval(seq, 8)").Check(testkit.Rows("8")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("10")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") tk.MustQuery("select setval(seq, 11)").Check(testkit.Rows("11")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") // set value can be bigger than maxvalue. tk.MustQuery("select setval(seq, 100)").Check(testkit.Rows("100")) err = tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") // test setval in second cache round. tk.MustExec("drop sequence if exists seq") @@ -460,22 +475,22 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select setval(seq, 20)").Check(testkit.Rows("20")) // the next value will not be base on next value. tk.MustQuery("select nextval(seq)").Check(testkit.Rows("25")) - sequenceTable := testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable := testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok := sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round := tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(95)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(95), end) + require.Equal(t, int64(0), round) // exhausted the sequence first round in cycle. tk.MustQuery("select setval(seq, 95)").Check(testkit.Rows("95")) // make sequence alloc the next batch. tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(91)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(91), end) + require.Equal(t, int64(1), round) tk.MustQuery("select setval(seq, 15)").Check(testkit.Rows("15")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("21")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("31")) @@ -485,41 +500,41 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select setval(seq, -20)").Check(testkit.Rows("")) tk.MustQuery("select setval(seq, 20)").Check(testkit.Rows("20")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-10")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(-6)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(-6), end) + require.Equal(t, int64(1), round) // test setval in negative-growth sequence. tk.MustExec("drop sequence if exists seq") tk.MustExec("create sequence seq increment -3 start 5 maxvalue 10 minvalue -10 cache 3 cycle") tk.MustQuery("select nextval(seq)").Check(testkit.Rows("5")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(-1)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(-1), end) + require.Equal(t, int64(0), round) // exhausted the sequence first cache batch. tk.MustQuery("select setval(seq, -2)").Check(testkit.Rows("-2")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-4")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(-10)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(-10), end) + require.Equal(t, int64(0), round) // exhausted the sequence second cache batch. tk.MustQuery("select setval(seq, -10)").Check(testkit.Rows("-10")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("10")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(4)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(4), end) + require.Equal(t, int64(1), round) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("7")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("4")) // test the sequence negative rebase. @@ -531,12 +546,12 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select setval(seq, 20)").Check(testkit.Rows("")) tk.MustQuery("select setval(seq, -20)").Check(testkit.Rows("-20")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("10")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(6)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(6), end) + require.Equal(t, int64(1), round) // test sequence lastval function. tk.MustExec("drop sequence if exists seq") @@ -559,34 +574,34 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustExec("create sequence seq increment 3 start 3 maxvalue 14 cache 3 cycle") tk.MustQuery("select lastval(seq)").Check(testkit.Rows("")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(9)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(9), end) + require.Equal(t, int64(0), round) // invalidate the current sequence cache. tk.MustQuery("select setval(seq, 10)").Check(testkit.Rows("10")) tk.MustQuery("select lastval(seq)").Check(testkit.Rows("3")) // trigger the next sequence cache. tk.MustQuery("select nextval(seq)").Check(testkit.Rows("12")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(14)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(14), end) + require.Equal(t, int64(0), round) // invalidate the current sequence cache. tk.MustQuery("select setval(seq, 13)").Check(testkit.Rows("13")) tk.MustQuery("select lastval(seq)").Check(testkit.Rows("12")) // trigger the next sequence cache. tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(7)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(7), end) + require.Equal(t, int64(1), round) tk.MustQuery("select lastval(seq)").Check(testkit.Rows("1")) // test lastval in negative-growth sequence cycle and cache. @@ -594,22 +609,22 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustExec("create sequence seq increment -3 start -2 maxvalue 10 minvalue -10 cache 3 cycle") tk.MustQuery("select lastval(seq)").Check(testkit.Rows("")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-2")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(-8)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(-8), end) + require.Equal(t, int64(0), round) // invalidate the current sequence cache. tk.MustQuery("select setval(seq, -8)").Check(testkit.Rows("-8")) tk.MustQuery("select lastval(seq)").Check(testkit.Rows("-2")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("10")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(4)) - c.Assert(round, Equals, int64(1)) + require.Equal(t, int64(4), end) + require.Equal(t, int64(1), round) tk.MustQuery("select lastval(seq)").Check(testkit.Rows("10")) tk.MustExec("drop sequence if exists seq") @@ -617,12 +632,12 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) tk.MustQuery("select setval(seq, -8)").Check(testkit.Rows("-8")) tk.MustQuery("select nextval(seq)").Check(testkit.Rows("-9")) - sequenceTable = testGetTableByName(c, tk.Se, "test", "seq") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok = sequenceTable.(*tables.TableCommon) - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() - c.Assert(end, Equals, int64(-10)) - c.Assert(round, Equals, int64(0)) + require.Equal(t, int64(-10), end) + require.Equal(t, int64(0), round) // Test the sequence seek formula will overflow Int64. tk.MustExec("drop sequence if exists seq") @@ -646,25 +661,25 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustExec("drop view if exists seq1") tk.MustExec("create table seq(a int)") _, err = tk.Exec("select nextval(seq)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq' is not SEQUENCE") _, err = tk.Exec("select lastval(seq)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq' is not SEQUENCE") _, err = tk.Exec("select setval(seq, 10)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq' is not SEQUENCE") tk.MustExec("create view seq1 as select * from seq") _, err = tk.Exec("select nextval(seq1)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq1' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq1' is not SEQUENCE") _, err = tk.Exec("select lastval(seq1)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq1' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq1' is not SEQUENCE") _, err = tk.Exec("select setval(seq1, 10)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.seq1' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.seq1' is not SEQUENCE") tk.MustExec("drop sequence if exists seq") tk.MustExec("drop table if exists seq") tk.MustExec("drop view if exists seq") @@ -685,10 +700,10 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { // test the current value already satisfied setval in other session. tk.MustExec("create sequence seq") tk.MustQuery("select setval(seq, 100)").Check(testkit.Rows("100")) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - tk1 := testkit.NewTestKit(c, s.store) - tk1.Se = se + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + tk1 := testkit.NewTestKit(t, store) + tk1.SetSession(se) tk1.MustExec("use test") tk1.MustQuery("select setval(seq, 50)").Check(testkit.Rows("")) tk1.MustQuery("select nextval(seq)").Check(testkit.Rows("101")) @@ -713,18 +728,21 @@ func (s *testSequenceSuite) TestSequenceFunction(c *C) { tk.MustExec("insert into t values(1),(2)") tk.MustQuery("select nextval(seq), t.a from t").Check(testkit.Rows("1 1", "2 2")) _, err = tk.Exec("select nextval(t), t.a from t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.t' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.t' is not SEQUENCE") _, err = tk.Exec("select nextval(seq), nextval(t), t.a from t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1347]'test.t' is not SEQUENCE") + require.Error(t, err) + require.EqualError(t, err, "[schema:1347]'test.t' is not SEQUENCE") tk.MustQuery("select nextval(seq)").Check(testkit.Rows("3")) tk.MustExec("drop sequence seq") tk.MustExec("drop table t") } -func (s *testSequenceSuite) TestInsertSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") tk.MustExec("drop table if exists t") @@ -788,11 +806,14 @@ func (s *testSequenceSuite) TestInsertSequence(c *C) { setSQL := "select setval(seq," + strconv.FormatInt(model.DefaultPositiveSequenceMaxValue+1, 10) + ")" tk.MustQuery(setSQL).Check(testkit.Rows("9223372036854775807")) err := tk.QueryToErr("select nextval(seq)") - c.Assert(err.Error(), Equals, "[table:4135]Sequence 'test.seq' has run out") + require.EqualError(t, err, "[table:4135]Sequence 'test.seq' has run out") } -func (s *testSequenceSuite) TestUnflodSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUnflodSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // test insert into select from. tk.MustExec("drop sequence if exists seq") @@ -856,9 +877,11 @@ func (s *testSequenceSuite) TestUnflodSequence(c *C) { // single insert consume: 50.498672ms // after this PR: // single insert consume: 33.213615ms -// Notice: use go test -check.b Benchmarkxxx to test it. -func (s *testSequenceSuite) BenchmarkInsertCacheDefaultExpr(c *C) { - tk := testkit.NewTestKit(c, s.store) +func BenchmarkInsertCacheDefaultExpr(b *testing.B) { + store, clean := testkit.CreateMockStore(b) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(b, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") tk.MustExec("drop table if exists t") @@ -872,14 +895,17 @@ func (s *testSequenceSuite) BenchmarkInsertCacheDefaultExpr(c *C) { sql += ",()" } } - c.ResetTimer() - for i := 0; i < c.N; i++ { + b.ResetTimer() + for i := 0; i < b.N; i++ { tk.MustExec(sql) } } -func (s *testSequenceSuite) TestSequenceFunctionPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceFunctionPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Test sequence function privilege. @@ -890,11 +916,11 @@ func (s *testSequenceSuite) TestSequenceFunctionPrivilege(c *C) { tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil)) + tk1.SetSession(se) // grant the myuser the create access to the sequence. tk.MustExec("grant insert on test.t to 'myuser'@'localhost'") @@ -902,22 +928,22 @@ func (s *testSequenceSuite) TestSequenceFunctionPrivilege(c *C) { // INSERT privilege required to use nextval. tk1.MustExec("use test") err = tk1.QueryToErr("select nextval(seq)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") + require.Error(t, err) + require.EqualError(t, err, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") _, err = tk1.Exec("insert into t values()") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") + require.Error(t, err) + require.EqualError(t, err, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") // SELECT privilege required to use lastval. err = tk1.QueryToErr("select lastval(seq)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1142]SELECT command denied to user 'myuser'@'localhost' for table 'seq'") + require.Error(t, err) + require.EqualError(t, err, "[expression:1142]SELECT command denied to user 'myuser'@'localhost' for table 'seq'") // INSERT privilege required to use setval. err = tk1.QueryToErr("select setval(seq, 10)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") + require.Error(t, err) + require.EqualError(t, err, "[expression:1142]INSERT command denied to user 'myuser'@'localhost' for table 'seq'") // grant the myuser the SELECT & UPDATE access to sequence seq. tk.MustExec("grant SELECT, INSERT on test.seq to 'myuser'@'localhost'") @@ -952,8 +978,11 @@ func (s *testSequenceSuite) TestSequenceFunctionPrivilege(c *C) { // So under current situation, TiDB will // [1]: forbid the new added column has sequence as it's default value. // [2]: allow the altered column with sequence as default value. -func (s *testSequenceSuite) TestSequenceDefaultLogic(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceDefaultLogic(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") @@ -981,18 +1010,21 @@ func (s *testSequenceSuite) TestSequenceDefaultLogic(c *C) { } // Close issue #17945, sequence cache shouldn't be negative. -func (s *testSequenceSuite) TestSequenceCacheShouldNotBeNegative(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceCacheShouldNotBeNegative(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") _, err := tk.Exec("create sequence seq cache -1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:4136]Sequence 'test.seq' values are conflicting") + require.Error(t, err) + require.EqualError(t, err, "[ddl:4136]Sequence 'test.seq' values are conflicting") _, err = tk.Exec("create sequence seq cache 0") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:4136]Sequence 'test.seq' values are conflicting") + require.Error(t, err) + require.EqualError(t, err, "[ddl:4136]Sequence 'test.seq' values are conflicting") // This will error because // 1: maxvalue = -1 by default @@ -1001,14 +1033,17 @@ func (s *testSequenceSuite) TestSequenceCacheShouldNotBeNegative(c *C) { // `seqInfo.CacheValue < (math.MaxInt64-absIncrement)/absIncrement` will // ensure there is enough value for one cache allocation at least. _, err = tk.Exec("create sequence seq INCREMENT -9223372036854775807 cache 1") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:4136]Sequence 'test.seq' values are conflicting") + require.Error(t, err) + require.EqualError(t, err, "[ddl:4136]Sequence 'test.seq' values are conflicting") tk.MustExec("create sequence seq cache 1") } -func (s *testSequenceSuite) TestAlterSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists seq") @@ -1065,8 +1100,11 @@ func (s *testSequenceSuite) TestAlterSequence(c *C) { tk.MustExec("drop sequence if exists seq") } -func (s *testSequenceSuite) TestAlterSequencePrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterSequencePrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + session.SetSchemaLease(600 * time.Millisecond) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists my_seq") tk.MustExec("create sequence my_seq") @@ -1075,27 +1113,27 @@ func (s *testSequenceSuite) TestAlterSequencePrivilege(c *C) { tk.MustExec("drop user if exists myuser@localhost") tk.MustExec("create user myuser@localhost") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "myuser", Hostname: "localhost"}, nil, nil)) + tk1.SetSession(se) // grant the myuser the access to database test. tk.MustExec("grant select on test.* to 'myuser'@'localhost'") tk1.MustExec("use test") _, err = tk1.Exec("alter sequence my_seq increment = 2") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1142]ALTER command denied to user 'myuser'@'localhost' for table 'my_seq'") + require.Error(t, err) + require.EqualError(t, err, "[planner:1142]ALTER command denied to user 'myuser'@'localhost' for table 'my_seq'") tk.MustExec("drop sequence if exists my_seq") } func TestDdl_AlterSequenceIssue31265(t *testing.T) { - store, clean := testkit2.CreateMockStore(t) + store, clean := testkit.CreateMockStore(t) defer clean() - tk := testkit2.NewTestKit(t, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create sequence seq cache=1 nocache") tk.MustQuery("show create sequence seq").Check(testkit.Rows("seq CREATE SEQUENCE `seq` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 nocache nocycle ENGINE=InnoDB")) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index b06bf67f824ca..130cae5e5086a 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -43,12 +43,13 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testutil" . "github.com/pingcap/tidb/util/testutil" "github.com/tikv/client-go/v2/testutils" ) @@ -59,6 +60,95 @@ var _ = SerialSuites(&testSerialSuite{}) // TODO(tangenta): Move all the parallel tests out of this file. var _ = Suite(&testIntegrationSuite7{&testIntegrationSuite{}}) +type testIntegrationSuite7 struct{ *testIntegrationSuite } + +type testIntegrationSuite struct { + lease time.Duration + cluster testutils.Cluster + store kv.Storage + dom *domain.Domain + ctx sessionctx.Context +} + +func setupIntegrationSuite(s *testIntegrationSuite, c *C) { + var err error + s.lease = 50 * time.Millisecond + ddl.SetWaitTimeWhenErrorOccurred(0) + + s.store, err = mockstore.NewMockStore( + mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockstore.BootstrapWithSingleStore(c) + s.cluster = c + }), + ) + c.Assert(err, IsNil) + session.SetSchemaLease(s.lease) + session.DisableStats4Test() + s.dom, err = session.BootstrapSession(s.store) + c.Assert(err, IsNil) + + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + s.ctx = se.(sessionctx.Context) + _, err = se.Execute(context.Background(), "create database test_db") + c.Assert(err, IsNil) +} + +func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show tables") + for _, tb := range r.Rows() { + tableName := tb[0] + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } +} + +func tearDownIntegrationSuite(s *testIntegrationSuite, c *C) { + s.dom.Close() + s.store.Close() +} + +func (s *testIntegrationSuite) SetUpSuite(c *C) { + setupIntegrationSuite(s, c) +} + +func (s *testIntegrationSuite) TearDownSuite(c *C) { + tearDownIntegrationSuite(s, c) +} + +type testMaxTableRowIDContext struct { + c *C + d ddl.DDL + tbl table.Table +} + +func newTestMaxTableRowIDContext(c *C, d ddl.DDL, tbl table.Table) *testMaxTableRowIDContext { + return &testMaxTableRowIDContext{ + c: c, + d: d, + tbl: tbl, + } +} + +func getMaxTableHandle(ctx *testMaxTableRowIDContext, store kv.Storage) (kv.Handle, bool) { + c := ctx.c + d := ctx.d + tbl := ctx.tbl + curVer, err := store.CurrentVersion(kv.GlobalTxnScope) + c.Assert(err, IsNil) + maxHandle, emptyTable, err := d.GetTableMaxHandle(curVer.Ver, tbl.(table.PhysicalTable)) + c.Assert(err, IsNil) + return maxHandle, emptyTable +} + +func checkGetMaxTableRowID(ctx *testMaxTableRowIDContext, store kv.Storage, expectEmpty bool, expectMaxHandle kv.Handle) { + c := ctx.c + maxHandle, emptyTable := getMaxTableHandle(ctx, store) + c.Assert(emptyTable, Equals, expectEmpty) + c.Assert(maxHandle, testutil.HandleEquals, expectMaxHandle) +} + type testSerialSuite struct { CommonHandleSuite store kv.Storage @@ -1521,9 +1611,6 @@ func (s *testSerialSuite) TestAutoRandomWithPreSplitRegion(c *C) { } func (s *testSerialSuite) TestModifyingColumn4NewCollations(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database dct") tk.MustExec("use dct") @@ -1557,8 +1644,6 @@ func (s *testSerialSuite) TestModifyingColumn4NewCollations(c *C) { } func (s *testSerialSuite) TestForbidUnsupportedCollations(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) mustGetUnsupportedCollation := func(sql string, coll string) { diff --git a/ddl/stat_test.go b/ddl/stat_test.go index ee68453849812..d2f52cb672c36 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -42,7 +42,7 @@ type testSerialStatSuiteToVerify struct { suite.Suite } -func TestSerialStatSuite(t *testing.T) { +func ExportTestSerialStatSuite(t *testing.T) { suite.Run(t, new(testSerialStatSuiteToVerify)) } diff --git a/ddl/table.go b/ddl/table.go index ac0e2c5860829..e53be47854b8c 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -89,18 +89,18 @@ func createTable(d *ddlCtx, t *meta.Meta, job *model.Job) (*model.TableInfo, err logutil.BgLogger().Info("Set TiFlash replica pd rule for partitioned table when creating", zap.Int64("tableID", tbInfo.ID)) if e := infosync.ConfigureTiFlashPDForPartitions(false, &pi.Definitions, replicaInfo.Count, &replicaInfo.LocationLabels); e != nil { job.State = model.JobStateCancelled - return tbInfo, errors.Trace(err) + return tbInfo, errors.Trace(e) } // Partitions that in adding mid-state. They have high priorities, so we should set accordingly pd rules. if e := infosync.ConfigureTiFlashPDForPartitions(true, &pi.AddingDefinitions, replicaInfo.Count, &replicaInfo.LocationLabels); e != nil { job.State = model.JobStateCancelled - return tbInfo, errors.Trace(err) + return tbInfo, errors.Trace(e) } } else { logutil.BgLogger().Info("Set TiFlash replica pd rule when creating", zap.Int64("tableID", tbInfo.ID)) if e := infosync.ConfigureTiFlashPDForTable(tbInfo.ID, replicaInfo.Count, &replicaInfo.LocationLabels); e != nil { job.State = model.JobStateCancelled - return tbInfo, errors.Trace(err) + return tbInfo, errors.Trace(e) } } } @@ -274,7 +274,7 @@ func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) } } -func onDropTableOrView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onDropTableOrView(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo, err := checkTableExistAndCancelNonExistJob(t, job, job.SchemaID) if err != nil { return ver, errors.Trace(err) @@ -686,13 +686,13 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro if e := infosync.ConfigureTiFlashPDForPartitions(true, &pi.Definitions, tblInfo.TiFlashReplica.Count, &tblInfo.TiFlashReplica.LocationLabels); e != nil { logutil.BgLogger().Error("ConfigureTiFlashPDForPartitions fails", zap.Error(err)) job.State = model.JobStateCancelled - return ver, errors.Trace(err) + return ver, errors.Trace(e) } } else { if e := infosync.ConfigureTiFlashPDForTable(newTableID, tblInfo.TiFlashReplica.Count, &tblInfo.TiFlashReplica.LocationLabels); e != nil { logutil.BgLogger().Error("ConfigureTiFlashPDForTable fails", zap.Error(err)) job.State = model.JobStateCancelled - return ver, errors.Trace(err) + return ver, errors.Trace(e) } } tblInfo.TiFlashReplica.AvailablePartitionIDs = nil @@ -1090,18 +1090,18 @@ func (w *worker) onSetTableFlashReplica(t *meta.Meta, job *model.Job) (ver int64 logutil.BgLogger().Info("Set TiFlash replica pd rule for partitioned table", zap.Int64("tableID", tblInfo.ID)) if e := infosync.ConfigureTiFlashPDForPartitions(false, &pi.Definitions, replicaInfo.Count, &replicaInfo.Labels); e != nil { job.State = model.JobStateCancelled - return ver, errors.Trace(err) + return ver, errors.Trace(e) } // Partitions that in adding mid-state. They have high priorities, so we should set accordingly pd rules. if e := infosync.ConfigureTiFlashPDForPartitions(true, &pi.AddingDefinitions, replicaInfo.Count, &replicaInfo.Labels); e != nil { job.State = model.JobStateCancelled - return ver, errors.Trace(err) + return ver, errors.Trace(e) } } else { logutil.BgLogger().Info("Set TiFlash replica pd rule", zap.Int64("tableID", tblInfo.ID)) if e := infosync.ConfigureTiFlashPDForTable(tblInfo.ID, replicaInfo.Count, &replicaInfo.Labels); e != nil { job.State = model.JobStateCancelled - return ver, errors.Trace(err) + return ver, errors.Trace(e) } } diff --git a/ddl/table_test.go b/ddl/table_test.go index d7e2650ff5ad9..b798409ae08b2 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -402,7 +402,7 @@ func ExportTestRenameTables(t *testing.T) { require.Equal(t, wantTblInfos[1].Name.L, "tt2") } -func TestCreateTables(t *testing.T) { +func ExportTestCreateTables(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) defer func() { diff --git a/ddl/util/dead_table_lock_checker.go b/ddl/util/dead_table_lock_checker.go index f05c1aa9534fa..b77be4d54412d 100644 --- a/ddl/util/dead_table_lock_checker.go +++ b/ddl/util/dead_table_lock_checker.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) diff --git a/ddl/util/main_test.go b/ddl/util/main_test.go index 9446758e36923..a3b8b8141e60d 100644 --- a/ddl/util/main_test.go +++ b/ddl/util/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), } goleak.VerifyTestMain(m, opts...) } diff --git a/ddl/util/syncer.go b/ddl/util/syncer.go index 287d49fa35c34..aa3cad4976fb4 100644 --- a/ddl/util/syncer.go +++ b/ddl/util/syncer.go @@ -31,9 +31,9 @@ import ( "github.com/pingcap/tidb/parser/terror" tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/zap" ) diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index 550a374badb98..75ca82c5bd698 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -30,10 +30,10 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/etcdserver" - "go.etcd.io/etcd/integration" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/server/v3/etcdserver" + "go.etcd.io/etcd/tests/v3/integration" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) @@ -45,6 +45,7 @@ func TestSyncerSimple(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) origin := CheckVersFirstWaitTime CheckVersFirstWaitTime = 0 diff --git a/ddl/util/util.go b/ddl/util/util.go index b5b77a5c10264..04ed74fec2b2e 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -18,6 +18,7 @@ import ( "context" "encoding/hex" "strings" + "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" @@ -176,11 +177,7 @@ func LoadGlobalVars(ctx context.Context, sctx sessionctx.Context, varNames []str paramNames = append(paramNames, name) } buf.WriteString(")") - stmt, err := e.ParseWithParams(ctx, true, buf.String(), paramNames...) - if err != nil { - return errors.Trace(err) - } - rows, _, err := e.ExecRestrictedStmt(ctx, stmt) + rows, _, err := e.ExecRestrictedSQL(ctx, nil, buf.String(), paramNames...) if err != nil { return errors.Trace(err) } @@ -194,3 +191,17 @@ func LoadGlobalVars(ctx context.Context, sctx sessionctx.Context, varNames []str } return nil } + +// GetTimeZone gets the session location's zone name and offset. +func GetTimeZone(sctx sessionctx.Context) (string, int) { + loc := sctx.GetSessionVars().Location() + name := loc.String() + if name != "" { + _, err := time.LoadLocation(name) + if err == nil { + return name, 0 + } + } + _, offset := time.Now().In(loc).Zone() + return "UTC", offset +} diff --git a/ddl/util_test.go b/ddl/util_test.go index d6a050703b232..71b9531bdfa26 100644 --- a/ddl/util_test.go +++ b/ddl/util_test.go @@ -170,7 +170,7 @@ func testCreateTable(t *testing.T, ctx sessionctx.Context, d *ddl, dbInfo *model Args: []interface{}{tblInfo}, } err := d.doDDLJob(ctx, job) - require.Nil(t, err) + require.NoError(t, err) v := getSchemaVer(t, ctx) tblInfo.State = model.StatePublic diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 9be2738da2251..6559ef8a0917e 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -462,7 +462,7 @@ func createSelectNormal(t *testing.T, batch, totalRows int, planIDs []int, sctx require.True(t, ok) require.Equal(t, "general", result.sqlType) require.Equal(t, "dag", result.label) - require.Equal(t, len(colTypes), result.rowLen) + require.Len(t, colTypes, result.rowLen) resp, ok := result.resp.(*mockResponse) require.True(t, ok) @@ -505,7 +505,7 @@ func createSelectStreaming(t *testing.T, batch, totalRows int) (*streamResult, [ require.NoError(t, err) result, ok := response.(*streamResult) require.True(t, ok) - require.Equal(t, len(colTypes), result.rowLen) + require.Len(t, colTypes, result.rowLen) resp, ok := result.resp.(*mockResponse) require.True(t, ok) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 13c83d6b3e2c4..2b1961de2d973 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -138,12 +138,6 @@ func (builder *RequestBuilder) SetPartitionsAndHandles(handles []kv.Handle) *Req return builder } -// SetIsolationLevel sets "IsolationLevel" for "kv.Request". -func (builder *RequestBuilder) SetIsolationLevel(level kv.IsoLevel) *RequestBuilder { - builder.Request.IsolationLevel = level - return builder -} - const estimatedRegionRowCount = 100000 // SetDAGRequest sets the request type to "ReqTypeDAG" and construct request data. @@ -250,7 +244,11 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req // Concurrency may be set to 1 by SetDAGRequest builder.Request.Concurrency = sv.DistSQLScanConcurrency() } - builder.Request.IsolationLevel = builder.getIsolationLevel() + if sv.StmtCtx.WeakConsistency { + builder.Request.IsolationLevel = kv.RC + } else { + builder.Request.IsolationLevel = builder.getIsolationLevel() + } builder.Request.NotFillCache = sv.StmtCtx.NotFillCache builder.Request.TaskID = sv.StmtCtx.TaskID builder.Request.Priority = builder.getKVPriority(sv) diff --git a/docs/design/2021-08-18-charsets.md b/docs/design/2021-08-18-charsets.md index 441f5b0917d6b..b048341f818e6 100644 --- a/docs/design/2021-08-18-charsets.md +++ b/docs/design/2021-08-18-charsets.md @@ -245,3 +245,72 @@ The third stage - Basically support all string-related functions already supported by TiDB. - Support the use of alter statement, support to modify the charset of the column. + +# Add New Charset + +### Encoding + +Define how character encode and decode, the main work is to implement [the encoding interface](https://github.com/pingcap/tidb/blob/7fd298d916c30ed0060db3581be58b448325819e/parser/charset/encoding.go#L65-L87) + +```golang +// Encoding provide encode/decode functions for a string with a specific charset. +type Encoding interface { + // Name is the name of the encoding. + Name() string + // Tp is the type of the encoding. + Tp() EncodingTp + // Peek returns the next char. + Peek(src []byte) []byte + // MbLen returns multiple byte length, if the next character is single byte, return 0. + MbLen(string) int + // IsValid checks whether the utf-8 bytes can be convert to valid string in current encoding. + IsValid(src []byte) bool + // Foreach iterates the characters in in current encoding. + Foreach(src []byte, op Op, fn func(from, to []byte, ok bool) bool) + // Transform map the bytes in src to dest according to Op. + // **the caller should initialize the dest if it wants to avoid memory alloc every time, or else it will always make a new one** + // **the returned array may be the alias of `src`, edit the returned array on your own risk** + Transform(dest *bytes.Buffer, src []byte, op Op) ([]byte, error) + // ToUpper change a string to uppercase. + ToUpper(src string) string + // ToLower change a string to lowercase. + ToLower(src string) string +} +``` +### Collation + +#### [New Collation](https://docs.pingcap.com/tidb/stable/character-set-and-collation#new-framework-for-collations) + +TiDB have no collation with the default config, all the collation will be ignored. After tidb 4.0, the collation framework is supported and uses a config to control it. After a new charset is added into TiDB, the related collation should also be added. + + +For example, `gb18030` charset have 3 different collations +```sql +| gb18030_bin | gb18030 | 249 | | Yes | 1 | PAD SPACE | +| gb18030_chinese_ci | gb18030 | 248 | Yes | Yes | 2 | PAD SPACE | +| gb18030_unicode_520_ci | gb18030 | 250 | | Yes | 8 | PAD SPACE | +``` + +We should at least implement `gb18030_bin` and `gb18030_chinese_ci` collations. + +If the collation framework is not used, the `gb18030_bin` should be the default collation, otherwise, the `gb18030_chinese_ci` should be the default collation. + +Add a new collation should implement the [Collator interface](https://github.com/pingcap/tidb/blob/7fd298d916c30ed0060db3581be58b448325819e/util/collate/collate.go#L62-L71) + +```golang +// Collator provides functionality for comparing strings for a given +// collation order. +type Collator interface { + // Compare returns an integer comparing the two strings. The result will be 0 if a == b, -1 if a < b, and +1 if a > b. + Compare(a, b string) int + // Key returns the collate key for str. If the collation is padding, make sure the PadLen >= len(rune[]str) in opt. + Key(str string) []byte + // Pattern get a collation-aware WildcardPattern. + Pattern() WildcardPattern +} +``` + +Then, add the charset and related collations into the support list to make tidb recognize it. + +### TikV +Many expressions have been pushed down to tikv, we should also make tikv support the [charset](https://github.com/tikv/tikv/blob/da1c069a2f9b5222ddbd32820a5a2ae1de50c78c/components/tidb_query_datatype/src/codec/collation/mod.rs#L98) and [collation](https://github.com/tikv/tikv/blob/da1c069a2f9b5222ddbd32820a5a2ae1de50c78c/components/tidb_query_datatype/src/codec/collation/mod.rs#L68). \ No newline at end of file diff --git a/docs/tidb_http_api.md b/docs/tidb_http_api.md index 1e87b1ee1ef57..b372ecded6788 100644 --- a/docs/tidb_http_api.md +++ b/docs/tidb_http_api.md @@ -542,6 +542,13 @@ timezone.* curl -X POST -d "tidb_enable_1pc=0" http://{TiDBIP}:10080/settings ``` +1. Enable/disable the mutation checker + + ```shell + curl -X POST -d "tidb_enable_mutation_checker=1" http://{TiDBIP}:10080/settings + curl -X POST -d "tidb_enable_mutation_checker=0" http://{TiDBIP}:10080/settings + ``` + 1. Get/Set the size of the Ballast Object ```shell diff --git a/domain/domain.go b/domain/domain.go index 11c9246551df5..054e9d537abe0 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -57,8 +57,9 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" "github.com/tikv/client-go/v2/txnkv/transaction" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" + pd "github.com/tikv/pd/client" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" atomicutil "go.uber.org/atomic" "go.uber.org/zap" "google.golang.org/grpc" @@ -363,12 +364,9 @@ func (do *Domain) InfoSyncer() *infosync.InfoSyncer { return do.info } -// NotifyGlobalConfigChange notify global config syncer to store the global config into PD(etcd). +// NotifyGlobalConfigChange notify global config syncer to store the global config into PD. func (do *Domain) NotifyGlobalConfigChange(name, value string) { - if do.globalCfgSyncer == nil { - return - } - do.globalCfgSyncer.Notify(name, value) + do.globalCfgSyncer.Notify(pd.GlobalConfigItem{Name: name, Value: value}) } // GetGlobalConfigSyncer exports for testing. @@ -704,7 +702,9 @@ func (do *Domain) Close() { } do.slowQuery.Close() - do.cancel() + if do.cancel != nil { + do.cancel() + } do.wg.Wait() do.sysSessionPool.Close() variable.UnregisterStatistics(do.bindHandle) @@ -815,7 +815,13 @@ func (do *Domain) Init(ddlLease time.Duration, sysExecutorFactory func(*Domain) if err != nil { return err } - do.globalCfgSyncer = globalconfigsync.NewGlobalConfigSyncer(do.etcdClient) + + var pdClient pd.Client + if store, ok := do.store.(kv.StorageWithPD); ok { + pdClient = store.GetPDClient() + } + do.globalCfgSyncer = globalconfigsync.NewGlobalConfigSyncer(pdClient) + err = do.ddl.SchemaSyncer().Init(ctx) if err != nil { return err @@ -878,9 +884,9 @@ type sessionPool struct { } } -func newSessionPool(cap int, factory pools.Factory) *sessionPool { +func newSessionPool(capacity int, factory pools.Factory) *sessionPool { return &sessionPool{ - resources: make(chan pools.Resource, cap), + resources: make(chan pools.Resource, capacity), factory: factory, } } diff --git a/domain/domain_test.go b/domain/domain_test.go index bed887c72133f..f1a9081509782 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -42,7 +42,7 @@ import ( "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" ) func TestInfo(t *testing.T) { @@ -50,6 +50,8 @@ func TestInfo(t *testing.T) { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) + if !unixSocketAvailable() { t.Skip("ETCD use ip:port as unix socket address, skip when it is unavailable.") } @@ -64,7 +66,7 @@ func TestInfo(t *testing.T) { mockStore := &mockEtcdBackend{ Storage: s, - pdAddrs: []string{cluster.Members[0].GRPCAddr()}} + pdAddrs: []string{cluster.Members[0].GRPCURL()}} ddlLease := 80 * time.Millisecond dom := NewDomain(mockStore, ddlLease, 0, 0, 0, mockFactory, nil) defer func() { diff --git a/domain/globalconfigsync/globalconfig.go b/domain/globalconfigsync/globalconfig.go index f6c6c86d276d9..5bbb6a260e3c8 100644 --- a/domain/globalconfigsync/globalconfig.go +++ b/domain/globalconfigsync/globalconfig.go @@ -17,58 +17,39 @@ package globalconfigsync import ( "context" - "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" + pd "github.com/tikv/pd/client" "go.uber.org/zap" ) -const ( - globalConfigPath = "/global/config/" - keyOpDefaultRetryCnt = 5 -) - -// GlobalConfigSyncer stores global config into etcd. +// GlobalConfigSyncer is used to sync pd global config. type GlobalConfigSyncer struct { - etcdCli *clientv3.Client - NotifyCh chan ConfigEntry + pd pd.Client + NotifyCh chan pd.GlobalConfigItem } -// NewGlobalConfigSyncer returns a new GlobalConfigSyncer. -func NewGlobalConfigSyncer(etcdCli *clientv3.Client) *GlobalConfigSyncer { +// NewGlobalConfigSyncer creates a GlobalConfigSyncer. +func NewGlobalConfigSyncer(p pd.Client) *GlobalConfigSyncer { return &GlobalConfigSyncer{ - etcdCli: etcdCli, - NotifyCh: make(chan ConfigEntry, 8), + pd: p, + NotifyCh: make(chan pd.GlobalConfigItem, 8), } } -// ConfigEntry contain the global config Name and Value. -type ConfigEntry struct { - Name string - Value string -} - -// Notify sends the config entry into notify channel. -func (c *GlobalConfigSyncer) Notify(name, value string) { - c.NotifyCh <- ConfigEntry{Name: name, Value: value} -} - -// StoreGlobalConfig stores the global config into etcd. -func (c *GlobalConfigSyncer) StoreGlobalConfig(ctx context.Context, entry ConfigEntry) error { - if c.etcdCli == nil { +// StoreGlobalConfig is used to store global config. +func (s *GlobalConfigSyncer) StoreGlobalConfig(ctx context.Context, item pd.GlobalConfigItem) error { + if s.pd == nil { return nil } - - key := globalConfigPath + entry.Name - err := util.PutKVToEtcd(ctx, c.etcdCli, keyOpDefaultRetryCnt, key, entry.Value) + err := s.pd.StoreGlobalConfig(ctx, []pd.GlobalConfigItem{item}) if err != nil { return err } - logutil.BgLogger().Info("store global config", zap.String("Name", entry.Name), zap.String("Value", entry.Value)) + logutil.BgLogger().Info("store global config", zap.String("name", item.Name), zap.String("value", item.Value)) return nil } -// SetEtcdClient exports for testing. -func (c *GlobalConfigSyncer) SetEtcdClient(etcdCli *clientv3.Client) { - c.etcdCli = etcdCli +// Notify pushes global config to internal channel and will be sync into pd's GlobalConfig. +func (s *GlobalConfigSyncer) Notify(globalConfigItem pd.GlobalConfigItem) { + s.NotifyCh <- globalConfigItem } diff --git a/domain/globalconfigsync/globalconfig_test.go b/domain/globalconfigsync/globalconfig_test.go index a1705adffea1d..565bc853a99b7 100644 --- a/domain/globalconfigsync/globalconfig_test.go +++ b/domain/globalconfigsync/globalconfig_test.go @@ -21,37 +21,53 @@ import ( "time" "github.com/pingcap/tidb/domain/globalconfigsync" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testbridge" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" + pd "github.com/tikv/pd/client" + "go.etcd.io/etcd/tests/v3/integration" "go.uber.org/goleak" ) func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) } func TestGlobalConfigSyncer(t *testing.T) { - syncer := globalconfigsync.NewGlobalConfigSyncer(nil) - syncer.Notify("a", "b") - require.Equal(t, len(syncer.NotifyCh), 1) - entry := <-syncer.NotifyCh - require.Equal(t, entry.Name, "a") - err := syncer.StoreGlobalConfig(context.Background(), entry) + if runtime.GOOS == "windows" { + t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") + } + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + err := store.Close() + require.NoError(t, err) + }() + client := store.(kv.StorageWithPD).GetPDClient() + syncer := globalconfigsync.NewGlobalConfigSyncer(client) + syncer.Notify(pd.GlobalConfigItem{Name: "a", Value: "b"}) + err = syncer.StoreGlobalConfig(context.Background(), <-syncer.NotifyCh) + require.NoError(t, err) + items, err := client.LoadGlobalConfig(context.Background(), []string{"a"}) require.NoError(t, err) + require.Equal(t, len(items), 1) + require.Equal(t, items[0].Name, "/global/config/a") + require.Equal(t, items[0].Value, "b") } func TestStoreGlobalConfig(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) + store, err := mockstore.NewMockStore() require.NoError(t, err) defer func() { @@ -63,32 +79,25 @@ func TestStoreGlobalConfig(t *testing.T) { require.NoError(t, err) defer domain.Close() - cluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) - defer cluster.Terminate(t) - domain.GetGlobalConfigSyncer().SetEtcdClient(cluster.RandClient()) - se, err := session.CreateSession4Test(store) require.NoError(t, err) _, err = se.Execute(context.Background(), "set @@global.tidb_enable_top_sql=1;") require.NoError(t, err) - for i := 0; i < 20; i++ { - resp, err := cluster.RandClient().Get(context.Background(), "/global/config/enable_resource_metering") + time.Sleep(100 * time.Millisecond) + client := + store.(kv.StorageWithPD).GetPDClient() + // enable top sql will be translated to enable_resource_metering + items, err := client.LoadGlobalConfig(context.Background(), []string{"enable_resource_metering"}) require.NoError(t, err) - require.NotNil(t, resp, nil) - - if len(resp.Kvs) == 0 { - // writing to ectd is async, so we should retry if not synced yet - time.Sleep(100 * time.Millisecond) + if len(items) == 1 && items[0].Value == "" { continue } - - require.Equal(t, len(resp.Kvs), 1) - require.Equal(t, resp.Kvs[0].Key, []byte("/global/config/enable_resource_metering")) - require.Equal(t, resp.Kvs[0].Value, []byte("true")) + require.Len(t, items, 1) + require.Equal(t, items[0].Name, "/global/config/enable_resource_metering") + require.Equal(t, items[0].Value, "true") return } - - require.Fail(t, "timeout for waiting etcd synced") + require.Fail(t, "timeout for waiting global config synced") } diff --git a/domain/infosync/info.go b/domain/infosync/info.go index d4eff4372668e..338a7da245573 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -51,8 +51,8 @@ import ( "github.com/pingcap/tidb/util/versioninfo" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/zap" ) @@ -1011,10 +1011,10 @@ func ConfigureTiFlashPDForTable(id int64, count uint64, locationLabels *[]string return errors.Trace(err) } ctx := context.Background() - logutil.BgLogger().Info("ConfigureTiFlashPDForTable", zap.Int64("tableID", id)) + logutil.BgLogger().Info("ConfigureTiFlashPDForTable", zap.Int64("tableID", id), zap.Uint64("count", count)) ruleNew := MakeNewRule(id, count, *locationLabels) if e := is.tiflashPlacementManager.SetPlacementRule(ctx, *ruleNew); e != nil { - return errors.Trace(err) + return errors.Trace(e) } return nil } @@ -1027,10 +1027,10 @@ func ConfigureTiFlashPDForPartitions(accel bool, definitions *[]model.PartitionD } ctx := context.Background() for _, p := range *definitions { - logutil.BgLogger().Info("ConfigureTiFlashPDForPartitions", zap.Int64("partID", p.ID), zap.Bool("accel", accel)) + logutil.BgLogger().Info("ConfigureTiFlashPDForPartitions", zap.Int64("partID", p.ID), zap.Bool("accel", accel), zap.Uint64("count", count)) ruleNew := MakeNewRule(p.ID, count, *locationLabels) if e := is.tiflashPlacementManager.SetPlacementRule(ctx, *ruleNew); e != nil { - return errors.Trace(err) + return errors.Trace(e) } if accel { e := is.tiflashPlacementManager.PostAccelerateSchedule(ctx, p.ID) diff --git a/domain/infosync/info_test.go b/domain/infosync/info_test.go index 50ecb0361d96e..cbfadd7c780e1 100644 --- a/domain/infosync/info_test.go +++ b/domain/infosync/info_test.go @@ -32,14 +32,14 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/testbridge" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" "go.uber.org/goleak" ) func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), } goleak.VerifyTestMain(m, opts...) } @@ -48,6 +48,7 @@ func TestTopology(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -223,7 +224,7 @@ func TestTiFlashManager(t *testing.T) { rule := MakeNewRule(1, 2, []string{"a"}) require.NoError(t, SetTiFlashPlacementRule(ctx, *rule)) rules, err := GetTiFlashGroupRules(ctx, "tiflash") - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, len(rules)) require.Equal(t, "table-1-r", rules[0].ID) require.Equal(t, 2, rules[0].Count) @@ -239,19 +240,19 @@ func TestTiFlashManager(t *testing.T) { // GetTiFlashStoresStat stats, err := GetTiFlashStoresStat(ctx) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, stats.Count) // DeleteTiFlashPlacementRule require.NoError(t, DeleteTiFlashPlacementRule(ctx, "tiflash", rule.ID)) rules, err = GetTiFlashGroupRules(ctx, "tiflash") - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 0, len(rules)) // ConfigureTiFlashPDForTable require.Nil(t, ConfigureTiFlashPDForTable(1, 2, &[]string{"a"})) rules, err = GetTiFlashGroupRules(ctx, "tiflash") - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 1, len(rules)) // ConfigureTiFlashPDForPartitions @@ -263,7 +264,7 @@ func TestTiFlashManager(t *testing.T) { }, }, 3, &[]string{}) rules, err = GetTiFlashGroupRules(ctx, "tiflash") - require.Nil(t, err) + require.NoError(t, err) // Have table 1 and 2 require.Equal(t, 2, len(rules)) z, ok = tiflash.SyncStatus[2] diff --git a/domain/infosync/tiflash_manager.go b/domain/infosync/tiflash_manager.go index d04db0e28ceb3..8243d32539777 100644 --- a/domain/infosync/tiflash_manager.go +++ b/domain/infosync/tiflash_manager.go @@ -70,6 +70,9 @@ func (m *TiFlashPDPlacementManager) Close(ctx context.Context) { // SetPlacementRule is a helper function to set placement rule. func (m *TiFlashPDPlacementManager) SetPlacementRule(ctx context.Context, rule placement.TiFlashRule) error { + if rule.Count == 0 { + return m.DeletePlacementRule(ctx, rule.GroupID, rule.ID) + } j, _ := json.Marshal(rule) buf := bytes.NewBuffer(j) res, err := doRequest(ctx, m.addrs, path.Join(pdapi.Config, "rule"), "POST", buf) @@ -184,6 +187,8 @@ func (m *TiFlashPDPlacementManager) GetStoresStat(ctx context.Context) (*helper. type mockTiFlashPlacementManager struct { sync.Mutex + // Set to nil if there is no need to set up a mock TiFlash server. + // Otherwise use NewMockTiFlash to create one. tiflash *MockTiFlash } @@ -241,6 +246,7 @@ func (m *mockTiFlashTableInfo) String() string { // MockTiFlash mocks a TiFlash, with necessary Pd support. type MockTiFlash struct { + sync.Mutex StatusAddr string StatusServer *httptest.Server SyncStatus map[int]mockTiFlashTableInfo @@ -250,7 +256,9 @@ type MockTiFlash struct { StartTime time.Time } -func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, string) { +func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() { + tiflash.Lock() + defer tiflash.Unlock() // mock TiFlash http server router := mux.NewRouter() server := httptest.NewServer(router) @@ -259,6 +267,8 @@ func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, stri statusAddrVec := strings.Split(statusAddr, ":") statusPort, _ := strconv.Atoi(statusAddrVec[1]) router.HandleFunc("/tiflash/sync-status/{tableid:\\d+}", func(w http.ResponseWriter, req *http.Request) { + tiflash.Lock() + defer tiflash.Unlock() params := mux.Vars(req) tableID, err := strconv.Atoi(params["tableid"]) if err != nil { @@ -276,11 +286,14 @@ func (tiflash *MockTiFlash) setUpMockTiFlashHTTPServer() (*httptest.Server, stri _, _ = w.Write([]byte(table.String())) }) router.HandleFunc("/config", func(w http.ResponseWriter, req *http.Request) { + tiflash.Lock() + defer tiflash.Unlock() s := fmt.Sprintf("{\n \"engine-store\": {\n \"http_port\": %v\n }\n}", statusPort) w.WriteHeader(http.StatusOK) _, _ = w.Write([]byte(s)) }) - return server, statusAddr + tiflash.StatusServer = server + tiflash.StatusAddr = statusAddr } // NewMockTiFlash creates a MockTiFlash with a mocked TiFlash server. @@ -294,19 +307,24 @@ func NewMockTiFlash() *MockTiFlash { TiflashDelay: 0, StartTime: time.Now(), } - server, addr := tiflash.setUpMockTiFlashHTTPServer() - tiflash.StatusAddr = addr - tiflash.StatusServer = server + tiflash.setUpMockTiFlashHTTPServer() return tiflash } // HandleSetPlacementRule is mock function for SetTiFlashPlacementRule. func (tiflash *MockTiFlash) HandleSetPlacementRule(rule placement.TiFlashRule) error { + tiflash.Lock() + defer tiflash.Unlock() if !tiflash.PdEnabled { - return errors.New("pd server is manually disabled, just quit") + logutil.BgLogger().Info("pd server is manually disabled, just quit") + return nil } - tiflash.GlobalTiFlashPlacementRules[rule.ID] = rule + if rule.Count == 0 { + delete(tiflash.GlobalTiFlashPlacementRules, rule.ID) + } else { + tiflash.GlobalTiFlashPlacementRules[rule.ID] = rule + } // Pd shall schedule TiFlash, we can mock here tid := 0 _, err := fmt.Sscanf(rule.ID, "table-%d-r", &tid) @@ -339,11 +357,15 @@ func (tiflash *MockTiFlash) HandleSetPlacementRule(rule placement.TiFlashRule) e // HandleDeletePlacementRule is mock function for DeleteTiFlashPlacementRule. func (tiflash *MockTiFlash) HandleDeletePlacementRule(group string, ruleID string) { + tiflash.Lock() + defer tiflash.Unlock() delete(tiflash.GlobalTiFlashPlacementRules, ruleID) } // HandleGetGroupRules is mock function for GetTiFlashGroupRules. func (tiflash *MockTiFlash) HandleGetGroupRules(group string) ([]placement.TiFlashRule, error) { + tiflash.Lock() + defer tiflash.Unlock() var result = make([]placement.TiFlashRule, 0) for _, item := range tiflash.GlobalTiFlashPlacementRules { result = append(result, item) @@ -353,6 +375,8 @@ func (tiflash *MockTiFlash) HandleGetGroupRules(group string) ([]placement.TiFla // HandlePostAccelerateSchedule is mock function for PostAccelerateSchedule func (tiflash *MockTiFlash) HandlePostAccelerateSchedule(endKey string) error { + tiflash.Lock() + defer tiflash.Unlock() tableID := helper.GetTiFlashTableIDFromEndKey(endKey) table, ok := tiflash.SyncStatus[int(tableID)] @@ -384,6 +408,8 @@ func (tiflash *MockTiFlash) HandleGetPDRegionRecordStats(_ int64) helper.PDRegio // HandleGetStoresStat is mock function for GetStoresStat. // It returns address of our mocked TiFlash server. func (tiflash *MockTiFlash) HandleGetStoresStat() *helper.StoresStat { + tiflash.Lock() + defer tiflash.Unlock() return &helper.StoresStat{ Count: 1, Stores: []helper.StoreStat{ @@ -407,12 +433,104 @@ func (tiflash *MockTiFlash) HandleGetStoresStat() *helper.StoresStat { } } +// Compare supposed rule, and we actually get from TableInfo +func isRuleMatch(rule placement.TiFlashRule, startKey string, endKey string, count int, labels []string) bool { + // Compute startKey + if rule.StartKeyHex == startKey && rule.EndKeyHex == endKey { + ok := false + for _, c := range rule.Constraints { + if c.Key == "engine" && len(c.Values) == 1 && c.Values[0] == "tiflash" && c.Op == placement.In { + ok = true + break + } + } + if !ok { + return false + } + + if len(rule.LocationLabels) == len(labels) { + for i, lb := range labels { + if lb != rule.LocationLabels[i] { + return false + } + } + } else { + return false + } + + if rule.Count != count { + return false + } + if rule.Role != placement.Learner { + return false + } + } else { + return false + } + return true +} + +// CheckPlacementRule find if a given rule precisely matches already set rules. +func (tiflash *MockTiFlash) CheckPlacementRule(rule placement.TiFlashRule) bool { + tiflash.Lock() + defer tiflash.Unlock() + for _, r := range tiflash.GlobalTiFlashPlacementRules { + if isRuleMatch(rule, r.StartKeyHex, r.EndKeyHex, r.Count, r.LocationLabels) { + return true + } + } + return false +} + +// GetPlacementRule find a rule by name. +func (tiflash *MockTiFlash) GetPlacementRule(ruleName string) (*placement.TiFlashRule, bool) { + tiflash.Lock() + defer tiflash.Unlock() + if r, ok := tiflash.GlobalTiFlashPlacementRules[ruleName]; ok { + p := r + return &p, ok + } + return nil, false +} + +// CleanPlacementRules cleans all placement rules. +func (tiflash *MockTiFlash) CleanPlacementRules() { + tiflash.Lock() + defer tiflash.Unlock() + tiflash.GlobalTiFlashPlacementRules = make(map[string]placement.TiFlashRule) +} + +// PlacementRulesLen gets length of all currently set placement rules. +func (tiflash *MockTiFlash) PlacementRulesLen() int { + tiflash.Lock() + defer tiflash.Unlock() + return len(tiflash.GlobalTiFlashPlacementRules) +} + +// GetTableSyncStatus returns table sync status by given tableID. +func (tiflash *MockTiFlash) GetTableSyncStatus(tableID int) (*mockTiFlashTableInfo, bool) { + tiflash.Lock() + defer tiflash.Unlock() + if r, ok := tiflash.SyncStatus[tableID]; ok { + p := r + return &p, ok + } + return nil, false +} + +// PdSwitch controls if pd is enabled. +func (tiflash *MockTiFlash) PdSwitch(enabled bool) { + tiflash.Lock() + defer tiflash.Unlock() + tiflash.PdEnabled = enabled +} + // SetPlacementRule is a helper function to set placement rule. func (m *mockTiFlashPlacementManager) SetPlacementRule(ctx context.Context, rule placement.TiFlashRule) error { m.Lock() defer m.Unlock() if m.tiflash == nil { - return errors.New("MockTiFlash is not accessible") + return nil } return m.tiflash.HandleSetPlacementRule(rule) } @@ -422,9 +540,9 @@ func (m *mockTiFlashPlacementManager) DeletePlacementRule(ctx context.Context, g m.Lock() defer m.Unlock() if m.tiflash == nil { - return errors.New("MockTiFlash is not accessible") + return nil } - logutil.BgLogger().Info("Remove TiFlash rule", zap.String("ID", ruleID)) + logutil.BgLogger().Info("Remove TiFlash rule", zap.String("ruleID", ruleID)) m.tiflash.HandleDeletePlacementRule(group, ruleID) return nil } @@ -434,7 +552,7 @@ func (m *mockTiFlashPlacementManager) GetGroupRules(ctx context.Context, group s m.Lock() defer m.Unlock() if m.tiflash == nil { - return nil, errors.New("MockTiFlash is not accessible") + return []placement.TiFlashRule{}, nil } return m.tiflash.HandleGetGroupRules(group) } @@ -444,7 +562,7 @@ func (m *mockTiFlashPlacementManager) PostAccelerateSchedule(ctx context.Context m.Lock() defer m.Unlock() if m.tiflash == nil { - return errors.New("MockTiFlash is not accessible") + return nil } endKey := tablecodec.EncodeTablePrefix(tableID + 1) endKey = codec.EncodeBytes([]byte{}, endKey) @@ -456,7 +574,7 @@ func (m *mockTiFlashPlacementManager) GetPDRegionRecordStats(ctx context.Context m.Lock() defer m.Unlock() if m.tiflash == nil { - return errors.New("MockTiFlash is not accessible") + return nil } *stats = m.tiflash.HandleGetPDRegionRecordStats(tableID) return nil @@ -467,7 +585,7 @@ func (m *mockTiFlashPlacementManager) GetStoresStat(ctx context.Context) (*helpe m.Lock() defer m.Unlock() if m.tiflash == nil { - return nil, errors.New("MockTiFlash is not accessible") + return nil, &MockTiFlashError{"MockTiFlash is not accessible"} } return m.tiflash.HandleGetStoresStat(), nil } @@ -476,7 +594,19 @@ func (m *mockTiFlashPlacementManager) GetStoresStat(ctx context.Context) (*helpe func (m *mockTiFlashPlacementManager) Close(ctx context.Context) { m.Lock() defer m.Unlock() + if m.tiflash == nil { + return + } if m.tiflash.StatusServer != nil { m.tiflash.StatusServer.Close() } } + +// MockTiFlashError represents MockTiFlash error +type MockTiFlashError struct { + Message string +} + +func (me *MockTiFlashError) Error() string { + return me.Message +} diff --git a/domain/main_test.go b/domain/main_test.go index aaa108626c8f3..f739274f1179b 100644 --- a/domain/main_test.go +++ b/domain/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/domain/plan_replayer_test.go b/domain/plan_replayer_test.go index f21abfd1fe5e8..aef54894489c8 100644 --- a/domain/plan_replayer_test.go +++ b/domain/plan_replayer_test.go @@ -29,10 +29,10 @@ func TestPlanReplayerGC(t *testing.T) { time := startTime.UnixNano() fileName := fmt.Sprintf("replayer_single_xxxxxx_%v.zip", time) err := os.MkdirAll(GetPlanReplayerDirName(), os.ModePerm) - require.Nil(t, err) + require.NoError(t, err) path := filepath.Join(GetPlanReplayerDirName(), fileName) zf, err := os.Create(path) - require.Nil(t, err) + require.NoError(t, err) zf.Close() handler := &planReplayer{} @@ -47,7 +47,7 @@ func TestPlanReplayerParseTime(t *testing.T) { nowTime := time.Now() name1 := fmt.Sprintf("replayer_single_xxxxxx_%v.zip", nowTime.UnixNano()) pt, err := parseTime(name1) - require.Nil(t, err) + require.NoError(t, err) require.True(t, pt.Equal(nowTime)) name2 := fmt.Sprintf("replayer_single_xxxxxx_%v1.zip", nowTime.UnixNano()) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index f67d7266392b6..687649c1f66b5 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -95,11 +95,7 @@ func (do *Domain) fetchTableValues(ctx sessionctx.Context) (map[string]string, e tableContents := make(map[string]string) // Copy all variables from the table to tableContents exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.Background(), true, `SELECT variable_name, variable_value FROM mysql.global_variables`) - if err != nil { - return tableContents, err - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, `SELECT variable_name, variable_value FROM mysql.global_variables`) if err != nil { return nil, err } diff --git a/dumpling/export/conn.go b/dumpling/export/conn.go new file mode 100644 index 0000000000000..c6723865b7003 --- /dev/null +++ b/dumpling/export/conn.go @@ -0,0 +1,108 @@ +// Copyright 2021 PingCAP, Inc. Licensed under Apache-2.0. + +package export + +import ( + "database/sql" + + "github.com/pingcap/errors" + "go.uber.org/zap" + + "github.com/pingcap/tidb/br/pkg/utils" + tcontext "github.com/pingcap/tidb/dumpling/context" +) + +// BaseConn wraps connection instance. +type BaseConn struct { + DBConn *sql.Conn + + backOffer backOfferResettable + rebuildConnFn func(*sql.Conn, bool) (*sql.Conn, error) +} + +func newBaseConn(conn *sql.Conn, shouldRetry bool, rebuildConnFn func(*sql.Conn, bool) (*sql.Conn, error)) *BaseConn { + baseConn := &BaseConn{DBConn: conn} + baseConn.backOffer = newRebuildConnBackOffer(shouldRetry) + if shouldRetry { + baseConn.rebuildConnFn = rebuildConnFn + } + return baseConn +} + +// QuerySQL defines query statement, and connect to real DB. +func (conn *BaseConn) QuerySQL(tctx *tcontext.Context, handleOneRow func(*sql.Rows) error, reset func(), query string, args ...interface{}) error { + retryTime := 0 + err := utils.WithRetry(tctx, func() (err error) { + retryTime++ + if retryTime > 1 && conn.rebuildConnFn != nil { + conn.DBConn, err = conn.rebuildConnFn(conn.DBConn, false) + if err != nil { + return + } + } + err = simpleQueryWithArgs(tctx, conn.DBConn, handleOneRow, query, args...) + if err != nil { + tctx.L().Info("cannot execute query", zap.Int("retryTime", retryTime), zap.String("sql", query), + zap.Any("args", args), zap.Error(err)) + reset() + return err + } + return nil + }, conn.backOffer) + conn.backOffer.Reset() + return err +} + +// QuerySQLWithColumns defines query statement, and connect to real DB and get results for special column names +func (conn *BaseConn) QuerySQLWithColumns(tctx *tcontext.Context, columns []string, query string, args ...interface{}) ([][]string, error) { + retryTime := 0 + var results [][]string + err := utils.WithRetry(tctx, func() (err error) { + retryTime++ + if retryTime > 1 && conn.rebuildConnFn != nil { + conn.DBConn, err = conn.rebuildConnFn(conn.DBConn, false) + if err != nil { + return + } + } + rows, err := conn.DBConn.QueryContext(tctx, query, args...) + if err != nil { + tctx.L().Info("cannot execute query", zap.Int("retryTime", retryTime), zap.String("sql", query), + zap.Any("args", args), zap.Error(err)) + return errors.Annotatef(err, "sql: %s", query) + } + results, err = GetSpecifiedColumnValuesAndClose(rows, columns...) + if err != nil { + tctx.L().Info("cannot execute query", zap.Int("retryTime", retryTime), zap.String("sql", query), + zap.Any("args", args), zap.Error(err)) + results = nil + return errors.Annotatef(err, "sql: %s", query) + } + return err + }, conn.backOffer) + conn.backOffer.Reset() + return results, err +} + +// ExecSQL defines exec statement, and connect to real DB. +func (conn *BaseConn) ExecSQL(tctx *tcontext.Context, canRetryFunc func(sql.Result, error) error, query string, args ...interface{}) error { + retryTime := 0 + err := utils.WithRetry(tctx, func() (err error) { + retryTime++ + if retryTime > 1 && conn.rebuildConnFn != nil { + conn.DBConn, err = conn.rebuildConnFn(conn.DBConn, false) + if err != nil { + return + } + } + res, err := conn.DBConn.ExecContext(tctx, query, args...) + if err = canRetryFunc(res, err); err != nil { + tctx.L().Info("cannot execute query", zap.Int("retryTime", retryTime), zap.String("sql", query), + zap.Any("args", args), zap.Error(err)) + return err + } + return nil + }, conn.backOffer) + conn.backOffer.Reset() + return err +} diff --git a/dumpling/export/dump.go b/dumpling/export/dump.go index 3f8008f7a09bd..4db8b22460c8a 100755 --- a/dumpling/export/dump.go +++ b/dumpling/export/dump.go @@ -52,7 +52,7 @@ type Dumper struct { dbHandle *sql.DB tidbPDClientForGC pd.Client - selectTiDBTableRegionFunc func(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) + selectTiDBTableRegionFunc func(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) totalTables int64 charsetAndDefaultCollationMap map[string]string } @@ -174,7 +174,7 @@ func (d *Dumper) Dump() (dumpErr error) { atomic.StoreInt64(&d.totalTables, int64(calculateTableCount(conf.Tables))) - rebuildConn := func(conn *sql.Conn) (*sql.Conn, error) { + rebuildConn := func(conn *sql.Conn, updateMeta bool) (*sql.Conn, error) { // make sure that the lock connection is still alive err1 := conCtrl.PingContext(tctx) if err1 != nil { @@ -188,7 +188,7 @@ func (d *Dumper) Dump() (dumpErr error) { } conn = newConn // renew the master status after connection. dm can't close safe-mode until dm reaches current pos - if conf.PosAfterConnect { + if updateMeta && conf.PosAfterConnect { err1 = m.recordGlobalMetaData(conn, conf.ServerInfo.ServerType, true) if err1 != nil { return conn, errors.Trace(err1) @@ -246,16 +246,17 @@ func (d *Dumper) Dump() (dumpErr error) { fmt.Printf("tidb_mem_quota_query == %s\n", s) } }) + baseConn := newBaseConn(metaConn, canRebuildConn(conf.Consistency, conf.TransactionalConsistency), rebuildConn) if conf.SQL == "" { - if err = d.dumpDatabases(writerCtx, metaConn, taskChan); err != nil && !errors.ErrorEqual(err, context.Canceled) { + if err = d.dumpDatabases(writerCtx, baseConn, taskChan); err != nil && !errors.ErrorEqual(err, context.Canceled) { return err } } else { - d.dumpSQL(writerCtx, metaConn, taskChan) + d.dumpSQL(writerCtx, baseConn, taskChan) } close(taskChan) - _ = metaConn.Close() + _ = baseConn.DBConn.Close() if err := wg.Wait(); err != nil { summary.CollectFailureUnit("dump table data", err) return errors.Trace(err) @@ -268,7 +269,7 @@ func (d *Dumper) Dump() (dumpErr error) { } func (d *Dumper) startWriters(tctx *tcontext.Context, wg *errgroup.Group, taskChan <-chan Task, - rebuildConnFn func(*sql.Conn) (*sql.Conn, error)) ([]*Writer, func(), error) { + rebuildConnFn func(*sql.Conn, bool) (*sql.Conn, error)) ([]*Writer, func(), error) { conf, pool := d.conf, d.dbHandle writers := make([]*Writer, conf.Threads) for i := 0; i < conf.Threads; i++ { @@ -311,14 +312,14 @@ func (d *Dumper) startWriters(tctx *tcontext.Context, wg *errgroup.Group, taskCh return writers, tearDown, nil } -func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskChan chan<- Task) error { +func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *BaseConn, taskChan chan<- Task) error { conf := d.conf allTables := conf.Tables // policy should be created before database // placement policy in other server type can be different, so we only handle the tidb server if conf.ServerInfo.ServerType == version.ServerTypeTiDB { - policyNames, err := ListAllPlacementPolicyNames(metaConn) + policyNames, err := ListAllPlacementPolicyNames(tctx, metaConn) if err != nil { errCause := errors.Cause(err) if mysqlErr, ok := errCause.(*mysql.MySQLError); ok && mysqlErr.Number == ErrNoSuchTable { @@ -329,7 +330,7 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC } } for _, policy := range policyNames { - createPolicySQL, err := ShowCreatePlacementPolicy(metaConn, policy) + createPolicySQL, err := ShowCreatePlacementPolicy(tctx, metaConn, policy) if err != nil { return errors.Trace(err) } @@ -345,7 +346,7 @@ func (d *Dumper) dumpDatabases(tctx *tcontext.Context, metaConn *sql.Conn, taskC parser1 := parser.New() for dbName, tables := range allTables { if !conf.NoSchemas { - createDatabaseSQL, err := ShowCreateDatabase(metaConn, dbName) + createDatabaseSQL, err := ShowCreateDatabase(tctx, metaConn, dbName) if err != nil { return errors.Trace(err) } @@ -531,14 +532,14 @@ func adjustColumnsCollation(tctx *tcontext.Context, createStmt *ast.CreateTableS } } -func (d *Dumper) dumpTableData(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error { +func (d *Dumper) dumpTableData(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, taskChan chan<- Task) error { conf := d.conf if conf.NoData { return nil } // Update total rows - fieldName, _ := pickupPossibleField(meta, conn) + fieldName, _ := pickupPossibleField(tctx, meta, conn) c := estimateCount(tctx, meta.DatabaseName(), meta.TableName(), conn, fieldName, conf) AddCounter(estimateTotalRowsCounter, conf.Labels, float64(c)) @@ -548,7 +549,7 @@ func (d *Dumper) dumpTableData(tctx *tcontext.Context, conn *sql.Conn, meta Tabl return d.concurrentDumpTable(tctx, conn, meta, taskChan) } -func (d *Dumper) buildConcatTask(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (*TaskTableData, error) { +func (d *Dumper) buildConcatTask(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) (*TaskTableData, error) { tableChan := make(chan Task, 128) errCh := make(chan error, 1) go func() { @@ -621,7 +622,7 @@ func (d *Dumper) dumpWholeTableDirectly(tctx *tcontext.Context, meta TableMeta, return nil } -func (d *Dumper) sequentialDumpTable(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error { +func (d *Dumper) sequentialDumpTable(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, taskChan chan<- Task) error { conf := d.conf if conf.ServerInfo.ServerType == version.ServerTypeTiDB { task, err := d.buildConcatTask(tctx, conn, meta) @@ -639,7 +640,7 @@ func (d *Dumper) sequentialDumpTable(tctx *tcontext.Context, conn *sql.Conn, met zap.String("database", meta.DatabaseName()), zap.String("table", meta.TableName())) } - orderByClause, err := buildOrderByClause(conf, conn, meta.DatabaseName(), meta.TableName(), meta.HasImplicitRowID()) + orderByClause, err := buildOrderByClause(tctx, conf, conn, meta.DatabaseName(), meta.TableName(), meta.HasImplicitRowID()) if err != nil { return err } @@ -647,7 +648,7 @@ func (d *Dumper) sequentialDumpTable(tctx *tcontext.Context, conn *sql.Conn, met } // concurrentDumpTable tries to split table into several chunks to dump -func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error { +func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, taskChan chan<- Task) error { conf := d.conf db, tbl := meta.DatabaseName(), meta.TableName() if conf.ServerInfo.ServerType == version.ServerTypeTiDB && @@ -664,12 +665,12 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met } } - orderByClause, err := buildOrderByClause(conf, conn, db, tbl, meta.HasImplicitRowID()) + orderByClause, err := buildOrderByClause(tctx, conf, conn, db, tbl, meta.HasImplicitRowID()) if err != nil { return err } - field, err := pickupPossibleField(meta, conn) + field, err := pickupPossibleField(tctx, meta, conn) if err != nil || field == "" { // skip split chunk logic if not found proper field tctx.L().Info("fallback to sequential dump due to no proper field. This won't influence the whole dump process", @@ -692,7 +693,7 @@ func (d *Dumper) concurrentDumpTable(tctx *tcontext.Context, conn *sql.Conn, met return d.dumpWholeTableDirectly(tctx, meta, taskChan, "", orderByClause, 0, 1) } - min, max, err := d.selectMinAndMaxIntValue(conn, db, tbl, field) + min, max, err := d.selectMinAndMaxIntValue(tctx, conn, db, tbl, field) if err != nil { tctx.L().Info("fallback to sequential dump due to cannot get bounding values. This won't influence the whole dump process", log.ShortError(err)) @@ -750,8 +751,8 @@ func (d *Dumper) sendTaskToChan(tctx *tcontext.Context, task Task, taskChan chan } } -func (d *Dumper) selectMinAndMaxIntValue(conn *sql.Conn, db, tbl, field string) (*big.Int, *big.Int, error) { - tctx, conf, zero := d.tctx, d.conf, &big.Int{} +func (d *Dumper) selectMinAndMaxIntValue(tctx *tcontext.Context, conn *BaseConn, db, tbl, field string) (*big.Int, *big.Int, error) { + conf, zero := d.conf, &big.Int{} query := fmt.Sprintf("SELECT MIN(`%s`),MAX(`%s`) FROM `%s`.`%s`", escapeString(field), escapeString(field), escapeString(db), escapeString(tbl)) if conf.Where != "" { @@ -761,8 +762,11 @@ func (d *Dumper) selectMinAndMaxIntValue(conn *sql.Conn, db, tbl, field string) var smin sql.NullString var smax sql.NullString - row := conn.QueryRowContext(tctx, query) - err := row.Scan(&smin, &smax) + err := conn.QuerySQL(tctx, func(rows *sql.Rows) error { + err := rows.Scan(&smin, &smax) + rows.Close() + return err + }, func() {}, query) if err != nil { return zero, zero, errors.Annotatef(err, "can't get min/max values to split chunks, query: %s", query) } @@ -783,7 +787,7 @@ func (d *Dumper) selectMinAndMaxIntValue(conn *sql.Conn, db, tbl, field string) return min, max, nil } -func (d *Dumper) concurrentDumpTiDBTables(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task) error { +func (d *Dumper) concurrentDumpTiDBTables(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, taskChan chan<- Task) error { db, tbl := meta.DatabaseName(), meta.TableName() var ( @@ -802,7 +806,7 @@ func (d *Dumper) concurrentDumpTiDBTables(tctx *tcontext.Context, conn *sql.Conn zap.String("database", db), zap.String("table", tbl)) var partitions []string if d.conf.ServerInfo.ServerVersion.Compare(*gcSafePointVersion) >= 0 { - partitions, err = GetPartitionNames(conn, db, tbl) + partitions, err = GetPartitionNames(tctx, conn, db, tbl) } if err == nil { if len(partitions) == 0 { @@ -818,7 +822,7 @@ func (d *Dumper) concurrentDumpTiDBTables(tctx *tcontext.Context, conn *sql.Conn return d.sendConcurrentDumpTiDBTasks(tctx, meta, taskChan, handleColNames, handleVals, "", 0, len(handleVals)+1) } -func (d *Dumper) concurrentDumpTiDBPartitionTables(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta, taskChan chan<- Task, partitions []string) error { +func (d *Dumper) concurrentDumpTiDBPartitionTables(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, taskChan chan<- Task, partitions []string) error { db, tbl := meta.DatabaseName(), meta.TableName() tctx.L().Debug("dumping TiDB tables with TABLE REGIONS for partition table", zap.String("database", db), zap.String("table", tbl), zap.Strings("partitions", partitions)) @@ -827,7 +831,7 @@ func (d *Dumper) concurrentDumpTiDBPartitionTables(tctx *tcontext.Context, conn totalChunk := 0 cachedHandleVals := make([][][]string, len(partitions)) - handleColNames, _, err := selectTiDBRowKeyFields(conn, meta, checkTiDBTableRegionPkFields) + handleColNames, _, err := selectTiDBRowKeyFields(tctx, conn, meta, checkTiDBTableRegionPkFields) if err != nil { return err } @@ -882,27 +886,28 @@ func (d *Dumper) L() log.Logger { return d.tctx.L() } -func selectTiDBTableSample(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { - pkFields, pkColTypes, err := selectTiDBRowKeyFields(conn, meta, nil) +func selectTiDBTableSample(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { + pkFields, pkColTypes, err := selectTiDBRowKeyFields(tctx, conn, meta, nil) if err != nil { return nil, nil, errors.Trace(err) } query := buildTiDBTableSampleQuery(pkFields, meta.DatabaseName(), meta.TableName()) - rows, err := conn.QueryContext(tctx, query) - if err != nil { - return nil, nil, errors.Trace(err) - } pkValNum := len(pkFields) - iter := newRowIter(rows, pkValNum) - defer iter.Close() + var iter SQLRowIter rowRec := MakeRowReceiver(pkColTypes) buf := new(bytes.Buffer) - for iter.HasNext() { + err = conn.QuerySQL(tctx, func(rows *sql.Rows) error { + if iter == nil { + iter = &rowIter{ + rows: rows, + args: make([]interface{}, pkValNum), + } + } err = iter.Decode(rowRec) if err != nil { - return nil, nil, errors.Trace(err) + return errors.Trace(err) } pkValRow := make([]string, 0, pkValNum) for _, rec := range rowRec.receivers { @@ -911,10 +916,21 @@ func selectTiDBTableSample(tctx *tcontext.Context, conn *sql.Conn, meta TableMet buf.Reset() } pkVals = append(pkVals, pkValRow) - iter.Next() + return nil + }, func() { + if iter != nil { + iter.Close() + iter = nil + } + rowRec = MakeRowReceiver(pkColTypes) + pkVals = pkVals[:0] + buf.Reset() + }, query) + if err == nil && iter != nil && iter.Error() != nil { + err = iter.Error() } - iter.Close() - return pkFields, pkVals, iter.Error() + + return pkFields, pkVals, err } func buildTiDBTableSampleQuery(pkFields []string, dbName, tblName string) string { @@ -927,11 +943,11 @@ func buildTiDBTableSampleQuery(pkFields []string, dbName, tblName string) string return fmt.Sprintf(template, pks, escapeString(dbName), escapeString(tblName), pks) } -func selectTiDBRowKeyFields(conn *sql.Conn, meta TableMeta, checkPkFields func([]string, []string) error) (pkFields, pkColTypes []string, err error) { +func selectTiDBRowKeyFields(tctx *tcontext.Context, conn *BaseConn, meta TableMeta, checkPkFields func([]string, []string) error) (pkFields, pkColTypes []string, err error) { if meta.HasImplicitRowID() { pkFields, pkColTypes = []string{"_tidb_rowid"}, []string{"BIGINT"} } else { - pkFields, pkColTypes, err = GetPrimaryKeyAndColumnTypes(conn, meta) + pkFields, pkColTypes, err = GetPrimaryKeyAndColumnTypes(tctx, conn, meta) if err == nil { if checkPkFields != nil { err = checkPkFields(pkFields, pkColTypes) @@ -952,8 +968,8 @@ func checkTiDBTableRegionPkFields(pkFields, pkColTypes []string) (err error) { return } -func selectTiDBTableRegion(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { - pkFields, _, err = selectTiDBRowKeyFields(conn, meta, checkTiDBTableRegionPkFields) +func selectTiDBTableRegion(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { + pkFields, _, err = selectTiDBRowKeyFields(tctx, conn, meta, checkTiDBTableRegionPkFields) if err != nil { return } @@ -968,7 +984,7 @@ func selectTiDBTableRegion(tctx *tcontext.Context, conn *sql.Conn, meta TableMet ) dbName, tableName := meta.DatabaseName(), meta.TableName() logger := tctx.L().With(zap.String("database", dbName), zap.String("table", tableName)) - err = simpleQueryWithArgs(conn, func(rows *sql.Rows) error { + err = conn.QuerySQL(tctx, func(rows *sql.Rows) error { rowID++ err = rows.Scan(&startKey, &decodedKey) if err != nil { @@ -994,38 +1010,33 @@ func selectTiDBTableRegion(tctx *tcontext.Context, conn *sql.Conn, meta TableMet pkVals = append(pkVals, []string{pkVal}) } return nil + }, func() { + pkFields = pkFields[:0] + pkVals = pkVals[:0] }, tableRegionSQL, dbName, tableName) return pkFields, pkVals, errors.Trace(err) } -func selectTiDBPartitionRegion(tctx *tcontext.Context, conn *sql.Conn, dbName, tableName, partition string) (pkVals [][]string, err error) { - var ( - rows *sql.Rows - startKeys []string - ) +func selectTiDBPartitionRegion(tctx *tcontext.Context, conn *BaseConn, dbName, tableName, partition string) (pkVals [][]string, err error) { + var startKeys [][]string const ( partitionRegionSQL = "SHOW TABLE `%s`.`%s` PARTITION(`%s`) REGIONS" regionRowKey = "r_" ) logger := tctx.L().With(zap.String("database", dbName), zap.String("table", tableName), zap.String("partition", partition)) - rows, err = conn.QueryContext(tctx, fmt.Sprintf(partitionRegionSQL, escapeString(dbName), escapeString(tableName), escapeString(partition))) - if err != nil { - err = errors.Trace(err) - return - } - startKeys, err = GetSpecifiedColumnValueAndClose(rows, "START_KEY") + startKeys, err = conn.QuerySQLWithColumns(tctx, []string{"START_KEY"}, fmt.Sprintf(partitionRegionSQL, escapeString(dbName), escapeString(tableName), escapeString(partition))) if err != nil { return } for rowID, startKey := range startKeys { - if rowID == 0 { + if rowID == 0 || len(startKey) != 1 { continue } - pkVal, err2 := extractTiDBRowIDFromDecodedKey(regionRowKey, startKey) + pkVal, err2 := extractTiDBRowIDFromDecodedKey(regionRowKey, startKey[0]) if err2 != nil { logger.Debug("show table region start key doesn't have rowID", - zap.Int("rowID", rowID), zap.String("startKey", startKey), log.ShortError(err2)) + zap.Int("rowID", rowID), zap.String("startKey", startKey[0]), zap.Error(err2)) } else { pkVals = append(pkVals, []string{pkVal}) } @@ -1087,9 +1098,9 @@ func prepareTableListToDump(tctx *tcontext.Context, conf *Config, db *sql.Conn) return nil } -func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db string, table *TableInfo) (TableMeta, error) { +func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *BaseConn, db string, table *TableInfo) (TableMeta, error) { tbl := table.Name - selectField, selectLen, err := buildSelectField(conn, db, tbl, conf.CompleteInsert) + selectField, selectLen, err := buildSelectField(tctx, conn, db, tbl, conf.CompleteInsert) if err != nil { return nil, err } @@ -1098,7 +1109,7 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db stri hasImplicitRowID bool ) if conf.ServerInfo.ServerType == version.ServerTypeTiDB { - hasImplicitRowID, err = SelectTiDBRowID(conn, db, tbl) + hasImplicitRowID, err = SelectTiDBRowID(tctx, conn, db, tbl) if err != nil { tctx.L().Info("check implicit rowID failed", zap.String("database", db), zap.String("table", tbl), log.ShortError(err)) } @@ -1106,9 +1117,9 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db stri // If all columns are generated if selectField == "" { - colTypes, err = GetColumnTypes(conn, "*", db, tbl) + colTypes, err = GetColumnTypes(tctx, conn, "*", db, tbl) } else { - colTypes, err = GetColumnTypes(conn, selectField, db, tbl) + colTypes, err = GetColumnTypes(tctx, conn, selectField, db, tbl) } if err != nil { return nil, err @@ -1132,7 +1143,7 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db stri } if table.Type == TableTypeView { viewName := table.Name - createTableSQL, createViewSQL, err1 := ShowCreateView(conn, db, viewName) + createTableSQL, createViewSQL, err1 := ShowCreateView(tctx, conn, db, viewName) if err1 != nil { return meta, err1 } @@ -1140,7 +1151,7 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db stri meta.showCreateView = createViewSQL return meta, nil } - createTableSQL, err := ShowCreateTable(conn, db, tbl) + createTableSQL, err := ShowCreateTable(tctx, conn, db, tbl) if err != nil { return nil, err } @@ -1148,7 +1159,7 @@ func dumpTableMeta(tctx *tcontext.Context, conf *Config, conn *sql.Conn, db stri return meta, nil } -func (d *Dumper) dumpSQL(tctx *tcontext.Context, metaConn *sql.Conn, taskChan chan<- Task) { +func (d *Dumper) dumpSQL(tctx *tcontext.Context, metaConn *BaseConn, taskChan chan<- Task) { conf := d.conf meta := &tableMeta{} data := newTableData(conf.SQL, 0, true) @@ -1439,7 +1450,7 @@ func (d *Dumper) renewSelectTableRegionFuncForLowerTiDB(tctx *tcontext.Context) // reference: https://github.com/pingcap/tidb/blob/c497d5c/dumpling/export/dump.go#L775 // To avoid this function continuously returning errors and confusing users because we fail to init this function at first, // selectTiDBTableRegionFunc is set to always return an ignorable error at first. - d.selectTiDBTableRegionFunc = func(_ *tcontext.Context, _ *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { + d.selectTiDBTableRegionFunc = func(_ *tcontext.Context, _ *BaseConn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { return nil, nil, errors.Annotatef(emptyHandleValsErr, "table: `%s`.`%s`", escapeString(meta.DatabaseName()), escapeString(meta.TableName())) } dbHandle, err := openDBFunc("mysql", conf.GetDSN("")) @@ -1505,8 +1516,8 @@ func (d *Dumper) renewSelectTableRegionFuncForLowerTiDB(tctx *tcontext.Context) } } - d.selectTiDBTableRegionFunc = func(tctx *tcontext.Context, conn *sql.Conn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { - pkFields, _, err = selectTiDBRowKeyFields(conn, meta, checkTiDBTableRegionPkFields) + d.selectTiDBTableRegionFunc = func(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) (pkFields []string, pkVals [][]string, err error) { + pkFields, _, err = selectTiDBRowKeyFields(tctx, conn, meta, checkTiDBTableRegionPkFields) if err != nil { return } diff --git a/dumpling/export/dump_test.go b/dumpling/export/dump_test.go index d74cb5432f09f..20d7831dca979 100644 --- a/dumpling/export/dump_test.go +++ b/dumpling/export/dump_test.go @@ -36,6 +36,7 @@ func TestDumpBlock(t *testing.T) { defer cancel() conn, err := db.Conn(tctx) require.NoError(t, err) + baseConn := newBaseConn(conn, true, nil) d := &Dumper{ tctx: tctx, @@ -52,14 +53,16 @@ func TestDumpBlock(t *testing.T) { time.Sleep(time.Second) return context.Canceled }) + writerCtx := tctx.WithContext(writingCtx) // simulate taskChan is full taskChan := make(chan Task, 1) taskChan <- &TaskDatabaseMeta{} d.conf.Tables = DatabaseTables{}.AppendTable(database, nil) d.conf.ServerInfo.ServerType = version.ServerTypeMySQL - require.ErrorIs(t, d.dumpDatabases(writerCtx, conn, taskChan), context.Canceled) require.ErrorIs(t, wg.Wait(), writerErr) + // if writerCtx is canceled , QuerySQL in `dumpDatabases` will return sqlmock.ErrCancelled + require.ErrorIs(t, d.dumpDatabases(writerCtx, baseConn, taskChan), sqlmock.ErrCancelled) } func TestDumpTableMeta(t *testing.T) { @@ -73,6 +76,7 @@ func TestDumpTableMeta(t *testing.T) { defer cancel() conn, err := db.Conn(tctx) require.NoError(t, err) + baseConn := newBaseConn(conn, true, nil) conf := DefaultConfig() conf.NoSchemas = true @@ -90,7 +94,7 @@ func TestDumpTableMeta(t *testing.T) { } mock.ExpectQuery(fmt.Sprintf("SELECT \\* FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows([]string{"id"}).AddRow(1)) - meta, err := dumpTableMeta(tctx, conf, conn, database, &TableInfo{Type: TableTypeBase, Name: table}) + meta, err := dumpTableMeta(tctx, conf, baseConn, database, &TableInfo{Type: TableTypeBase, Name: table}) require.NoError(t, err) require.Equal(t, database, meta.DatabaseName()) require.Equal(t, table, meta.TableName()) diff --git a/dumpling/export/retry.go b/dumpling/export/retry.go index 13f4931d72a0c..3facc7e296189 100644 --- a/dumpling/export/retry.go +++ b/dumpling/export/retry.go @@ -11,6 +11,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "go.uber.org/zap" + "github.com/pingcap/tidb/br/pkg/utils" tcontext "github.com/pingcap/tidb/dumpling/context" ) @@ -23,9 +24,14 @@ const ( ErrNoSuchTable uint16 = 1146 ) -func newDumpChunkBackoffer(shouldRetry bool) *dumpChunkBackoffer { // revive:disable-line:flag-parameter +type backOfferResettable interface { + utils.Backoffer + Reset() +} + +func newRebuildConnBackOffer(shouldRetry bool) backOfferResettable { // revive:disable-line:flag-parameter if !shouldRetry { - return &dumpChunkBackoffer{ + return &noopBackoffer{ attempt: 1, } } @@ -47,10 +53,6 @@ func (b *dumpChunkBackoffer) NextBackoff(err error) time.Duration { if _, ok := err.(*mysql.MySQLError); ok && !dbutil.IsRetryableError(err) { b.attempt = 0 return 0 - } else if _, ok := err.(*writerError); ok { - // the uploader writer's retry logic is already done in aws client. needn't retry here - b.attempt = 0 - return 0 } b.delayTime = 2 * b.delayTime b.attempt-- @@ -64,6 +66,28 @@ func (b *dumpChunkBackoffer) Attempt() int { return b.attempt } +func (b *dumpChunkBackoffer) Reset() { + b.attempt = dumpChunkRetryTime + b.delayTime = dumpChunkWaitInterval +} + +type noopBackoffer struct { + attempt int +} + +func (b *noopBackoffer) NextBackoff(err error) time.Duration { + b.attempt-- + return time.Duration(0) +} + +func (b *noopBackoffer) Attempt() int { + return b.attempt +} + +func (b *noopBackoffer) Reset() { + b.attempt = 1 +} + func newLockTablesBackoffer(tctx *tcontext.Context, blockList map[string]map[string]interface{}) *lockTablesBackoffer { return &lockTablesBackoffer{ tctx: tctx, diff --git a/dumpling/export/sql.go b/dumpling/export/sql.go index 43722b448eb38..fc70d5a750215 100644 --- a/dumpling/export/sql.go +++ b/dumpling/export/sql.go @@ -16,11 +16,14 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "go.uber.org/multierr" "go.uber.org/zap" dbconfig "github.com/pingcap/tidb/config" tcontext "github.com/pingcap/tidb/dumpling/context" "github.com/pingcap/tidb/dumpling/log" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/helper" ) @@ -57,59 +60,63 @@ func ShowTables(db *sql.Conn) ([]string, error) { // ShowCreateDatabase constructs the create database SQL for a specified database // returns (createDatabaseSQL, error) -func ShowCreateDatabase(db *sql.Conn, database string) (string, error) { +func ShowCreateDatabase(tctx *tcontext.Context, db *BaseConn, database string) (string, error) { var oneRow [2]string handleOneRow := func(rows *sql.Rows) error { return rows.Scan(&oneRow[0], &oneRow[1]) } query := fmt.Sprintf("SHOW CREATE DATABASE `%s`", escapeString(database)) - err := simpleQuery(db, query, handleOneRow) - if mysqlErr, ok := errors.Cause(err).(*mysql.MySQLError); ok { - // Falling back to simple create statement for MemSQL/SingleStore, because of this: - // ERROR 1706 (HY000): Feature 'SHOW CREATE DATABASE' is not supported by MemSQL. - if mysqlErr.Number == 1706 { - return fmt.Sprintf("CREATE DATABASE `%s`", escapeString(database)), nil + err := db.QuerySQL(tctx, handleOneRow, func() { + oneRow[0], oneRow[1] = "", "" + }, query) + if multiErrs := multierr.Errors(err); len(multiErrs) > 0 { + for _, multiErr := range multiErrs { + if mysqlErr, ok := errors.Cause(multiErr).(*mysql.MySQLError); ok { + // Falling back to simple create statement for MemSQL/SingleStore, because of this: + // ERROR 1706 (HY000): Feature 'SHOW CREATE DATABASE' is not supported by MemSQL. + if strings.Contains(mysqlErr.Error(), "SHOW CREATE DATABASE") { + return fmt.Sprintf("CREATE DATABASE `%s`", escapeString(database)), nil + } + } } } - if err != nil { - return "", errors.Annotatef(err, "sql: %s", query) - } - return oneRow[1], nil + return oneRow[1], err } // ShowCreateTable constructs the create table SQL for a specified table // returns (createTableSQL, error) -func ShowCreateTable(db *sql.Conn, database, table string) (string, error) { +func ShowCreateTable(tctx *tcontext.Context, db *BaseConn, database, table string) (string, error) { var oneRow [2]string handleOneRow := func(rows *sql.Rows) error { return rows.Scan(&oneRow[0], &oneRow[1]) } query := fmt.Sprintf("SHOW CREATE TABLE `%s`.`%s`", escapeString(database), escapeString(table)) - err := simpleQuery(db, query, handleOneRow) + err := db.QuerySQL(tctx, handleOneRow, func() { + oneRow[0], oneRow[1] = "", "" + }, query) if err != nil { - return "", errors.Annotatef(err, "sql: %s", query) + return "", err } return oneRow[1], nil } // ShowCreatePlacementPolicy constructs the create policy SQL for a specified table // returns (createPoilicySQL, error) -func ShowCreatePlacementPolicy(db *sql.Conn, policy string) (string, error) { +func ShowCreatePlacementPolicy(tctx *tcontext.Context, db *BaseConn, policy string) (string, error) { var oneRow [2]string handleOneRow := func(rows *sql.Rows) error { return rows.Scan(&oneRow[0], &oneRow[1]) } query := fmt.Sprintf("SHOW CREATE PLACEMENT POLICY `%s`", escapeString(policy)) - err := simpleQuery(db, query, handleOneRow) - if err != nil { - return "", errors.Annotatef(err, "sql: %s", query) - } - return oneRow[1], nil + err := db.QuerySQL(tctx, handleOneRow, func() { + oneRow[0], oneRow[1] = "", "" + }, query) + return oneRow[1], err } // ShowCreateView constructs the create view SQL for a specified view // returns (createFakeTableSQL, createViewSQL, error) -func ShowCreateView(db *sql.Conn, database, view string) (createFakeTableSQL string, createRealViewSQL string, err error) { +func ShowCreateView(tctx *tcontext.Context, db *BaseConn, database, view string) (createFakeTableSQL string, createRealViewSQL string, err error) { var fieldNames []string handleFieldRow := func(rows *sql.Rows) error { var oneRow [6]sql.NullString @@ -130,9 +137,11 @@ func ShowCreateView(db *sql.Conn, database, view string) (createFakeTableSQL str // Build createTableSQL query := fmt.Sprintf("SHOW FIELDS FROM `%s`.`%s`", escapeString(database), escapeString(view)) - err = simpleQuery(db, query, handleFieldRow) + err = db.QuerySQL(tctx, handleFieldRow, func() { + fieldNames = []string{} + }, query) if err != nil { - return "", "", errors.Annotatef(err, "sql: %s", query) + return "", "", err } fmt.Fprintf(&createTableSQL, "CREATE TABLE `%s`(\n", escapeString(view)) createTableSQL.WriteString(strings.Join(fieldNames, ",\n")) @@ -142,9 +151,13 @@ func ShowCreateView(db *sql.Conn, database, view string) (createFakeTableSQL str fmt.Fprintf(&createViewSQL, "DROP TABLE IF EXISTS `%s`;\n", escapeString(view)) fmt.Fprintf(&createViewSQL, "DROP VIEW IF EXISTS `%s`;\n", escapeString(view)) query = fmt.Sprintf("SHOW CREATE VIEW `%s`.`%s`", escapeString(database), escapeString(view)) - err = simpleQuery(db, query, handleOneRow) + err = db.QuerySQL(tctx, handleOneRow, func() { + for i := range oneRow { + oneRow[i] = "" + } + }, query) if err != nil { - return "", "", errors.Annotatef(err, "sql: %s", query) + return "", "", err } // The result for `show create view` SQL // mysql> show create view v1; @@ -203,7 +216,7 @@ func ListAllDatabasesTables(tctx *tcontext.Context, db *sql.Conn, databaseNames for _, schema := range databaseNames { dbTables[schema] = make([]*TableInfo, 0) } - if err = simpleQueryWithArgs(db, func(rows *sql.Rows) error { + if err = simpleQueryWithArgs(tctx, db, func(rows *sql.Rows) error { var ( sqlAvgRowLength sql.NullInt64 err2 error @@ -234,7 +247,7 @@ func ListAllDatabasesTables(tctx *tcontext.Context, db *sql.Conn, databaseNames dbTables[schema] = make([]*TableInfo, 0) query := fmt.Sprintf("SHOW FULL TABLES FROM `%s` WHERE %s", escapeString(schema), strings.Join(tableTypeConditions, " OR ")) - if err = simpleQueryWithArgs(db, func(rows *sql.Rows) error { + if err = simpleQueryWithArgs(tctx, db, func(rows *sql.Rows) error { var err2 error if err2 = rows.Scan(&table, &tableTypeStr); err != nil { return errors.Trace(err2) @@ -294,23 +307,21 @@ func ListAllDatabasesTables(tctx *tcontext.Context, db *sql.Conn, databaseNames return dbTables, nil } -func ListAllPlacementPolicyNames(db *sql.Conn) ([]string, error) { +func ListAllPlacementPolicyNames(tctx *tcontext.Context, db *BaseConn) ([]string, error) { var policyList []string var policy string const query = "select distinct policy_name from information_schema.placement_policies where policy_name is not null;" - rows, err := db.QueryContext(context.Background(), query) - if err != nil { - return policyList, errors.Annotatef(err, "sql: %s", query) - } - defer rows.Close() - for rows.Next() { + err := db.QuerySQL(tctx, func(rows *sql.Rows) error { err := rows.Scan(&policy) if err != nil { - return policyList, errors.Trace(err) + return errors.Trace(err) } policyList = append(policyList, policy) - } - return policyList, errors.Annotatef(rows.Err(), "sql: %s", query) + return nil + }, func() { + policyList = policyList[:0] + }, query) + return policyList, errors.Annotatef(err, "sql: %s", query) } // SelectVersion gets the version information from the database server @@ -370,14 +381,14 @@ func buildSelectQuery(database, table, fields, partition, where, orderByClause s return query.String() } -func buildOrderByClause(conf *Config, db *sql.Conn, database, table string, hasImplicitRowID bool) (string, error) { // revive:disable-line:flag-parameter +func buildOrderByClause(tctx *tcontext.Context, conf *Config, db *BaseConn, database, table string, hasImplicitRowID bool) (string, error) { // revive:disable-line:flag-parameter if !conf.SortByPk { return "", nil } if hasImplicitRowID { return orderByTiDBRowID, nil } - cols, err := GetPrimaryKeyColumns(db, database, table) + cols, err := GetPrimaryKeyColumns(tctx, db, database, table) if err != nil { return "", errors.Trace(err) } @@ -385,18 +396,22 @@ func buildOrderByClause(conf *Config, db *sql.Conn, database, table string, hasI } // SelectTiDBRowID checks whether this table has _tidb_rowid column -func SelectTiDBRowID(db *sql.Conn, database, table string) (bool, error) { - const errBadFieldCode = 1054 +func SelectTiDBRowID(tctx *tcontext.Context, db *BaseConn, database, table string) (bool, error) { tiDBRowIDQuery := fmt.Sprintf("SELECT _tidb_rowid from `%s`.`%s` LIMIT 1", escapeString(database), escapeString(table)) - _, err := db.ExecContext(context.Background(), tiDBRowIDQuery) - if err != nil { - errMsg := strings.ToLower(err.Error()) - if strings.Contains(errMsg, strconv.Itoa(errBadFieldCode)) { - return false, nil + hasImplictRowID := false + err := db.ExecSQL(tctx, func(_ sql.Result, err error) error { + if err != nil { + hasImplictRowID = false + errMsg := strings.ToLower(err.Error()) + if strings.Contains(errMsg, fmt.Sprintf("%d", errno.ErrBadField)) { + return nil + } + return errors.Annotatef(err, "sql: %s", tiDBRowIDQuery) } - return false, errors.Annotatef(err, "sql: %s", tiDBRowIDQuery) - } - return true, nil + hasImplictRowID = true + return nil + }, tiDBRowIDQuery) + return hasImplictRowID, err } // GetSuitableRows gets suitable rows for each table @@ -417,26 +432,35 @@ func GetSuitableRows(avgRowLength uint64) uint64 { } // GetColumnTypes gets *sql.ColumnTypes from a specified table -func GetColumnTypes(db *sql.Conn, fields, database, table string) ([]*sql.ColumnType, error) { +func GetColumnTypes(tctx *tcontext.Context, db *BaseConn, fields, database, table string) ([]*sql.ColumnType, error) { query := fmt.Sprintf("SELECT %s FROM `%s`.`%s` LIMIT 1", fields, escapeString(database), escapeString(table)) - rows, err := db.QueryContext(context.Background(), query) + var colTypes []*sql.ColumnType + err := db.QuerySQL(tctx, func(rows *sql.Rows) error { + var err error + colTypes, err = rows.ColumnTypes() + if err == nil { + err = rows.Close() + } + failpoint.Inject("ChaosBrokenMetaConn", func(_ failpoint.Value) { + failpoint.Return(errors.New("connection is closed")) + }) + return errors.Annotatef(err, "sql: %s", query) + }, func() { + colTypes = nil + }, query) if err != nil { - return nil, errors.Annotatef(err, "sql: %s", query) - } - defer rows.Close() - if err = rows.Err(); err != nil { - return nil, errors.Annotatef(err, "sql: %s", query) + return nil, err } - return rows.ColumnTypes() + return colTypes, nil } // GetPrimaryKeyAndColumnTypes gets all primary columns and their types in ordinal order -func GetPrimaryKeyAndColumnTypes(conn *sql.Conn, meta TableMeta) ([]string, []string, error) { +func GetPrimaryKeyAndColumnTypes(tctx *tcontext.Context, conn *BaseConn, meta TableMeta) ([]string, []string, error) { var ( colNames, colTypes []string err error ) - colNames, err = GetPrimaryKeyColumns(conn, meta.DatabaseName(), meta.TableName()) + colNames, err = GetPrimaryKeyColumns(tctx, conn, meta.DatabaseName(), meta.TableName()) if err != nil { return nil, nil, err } @@ -449,16 +473,13 @@ func GetPrimaryKeyAndColumnTypes(conn *sql.Conn, meta TableMeta) ([]string, []st } // GetPrimaryKeyColumns gets all primary columns in ordinal order -func GetPrimaryKeyColumns(db *sql.Conn, database, table string) ([]string, error) { +func GetPrimaryKeyColumns(tctx *tcontext.Context, db *BaseConn, database, table string) ([]string, error) { priKeyColsQuery := fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", escapeString(database), escapeString(table)) - rows, err := db.QueryContext(context.Background(), priKeyColsQuery) + results, err := db.QuerySQLWithColumns(tctx, []string{"KEY_NAME", "COLUMN_NAME"}, priKeyColsQuery) if err != nil { - return nil, errors.Annotatef(err, "sql: %s", priKeyColsQuery) - } - results, err := GetSpecifiedColumnValuesAndClose(rows, "KEY_NAME", "COLUMN_NAME") - if err != nil { - return nil, errors.Annotatef(err, "sql: %s", priKeyColsQuery) + return nil, err } + cols := make([]string, 0, len(results)) for _, oneRow := range results { keyName, columnName := oneRow[0], oneRow[1] @@ -472,17 +493,13 @@ func GetPrimaryKeyColumns(db *sql.Conn, database, table string) ([]string, error // getNumericIndex picks up indices according to the following priority: // primary key > unique key with the smallest count > key with the max cardinality // primary key with multi cols is before unique key with single col because we will sort result by primary keys -func getNumericIndex(db *sql.Conn, meta TableMeta) (string, error) { +func getNumericIndex(tctx *tcontext.Context, db *BaseConn, meta TableMeta) (string, error) { database, table := meta.DatabaseName(), meta.TableName() colName2Type := string2Map(meta.ColumnNames(), meta.ColumnTypes()) keyQuery := fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", escapeString(database), escapeString(table)) - rows, err := db.QueryContext(context.Background(), keyQuery) - if err != nil { - return "", errors.Annotatef(err, "sql: %s", keyQuery) - } - results, err := GetSpecifiedColumnValuesAndClose(rows, "NON_UNIQUE", "SEQ_IN_INDEX", "KEY_NAME", "COLUMN_NAME", "CARDINALITY") + results, err := db.QuerySQLWithColumns(tctx, []string{"NON_UNIQUE", "SEQ_IN_INDEX", "KEY_NAME", "COLUMN_NAME", "CARDINALITY"}, keyQuery) if err != nil { - return "", errors.Annotatef(err, "sql: %s", keyQuery) + return "", err } type keyColumnPair struct { colName string @@ -838,20 +855,14 @@ func createConnWithConsistency(ctx context.Context, db *sql.DB, repeatableRead b // buildSelectField returns the selecting fields' string(joined by comma(`,`)), // and the number of writable fields. -func buildSelectField(db *sql.Conn, dbName, tableName string, completeInsert bool) (string, int, error) { // revive:disable-line:flag-parameter +func buildSelectField(tctx *tcontext.Context, db *BaseConn, dbName, tableName string, completeInsert bool) (string, int, error) { // revive:disable-line:flag-parameter query := fmt.Sprintf("SHOW COLUMNS FROM `%s`.`%s`", escapeString(dbName), escapeString(tableName)) - rows, err := db.QueryContext(context.Background(), query) + results, err := db.QuerySQLWithColumns(tctx, []string{"FIELD", "EXTRA"}, query) if err != nil { - return "", 0, errors.Annotatef(err, "sql: %s", query) + return "", 0, err } - defer rows.Close() availableFields := make([]string, 0) - hasGenerateColumn := false - results, err := GetSpecifiedColumnValuesAndClose(rows, "FIELD", "EXTRA") - if err != nil { - return "", 0, errors.Annotatef(err, "sql: %s", query) - } for _, oneRow := range results { fieldName, extra := oneRow[0], oneRow[1] switch extra { @@ -1052,34 +1063,41 @@ func (o *oneStrColumnTable) handleOneRow(rows *sql.Rows) error { return nil } -func simpleQuery(conn *sql.Conn, sql string, handleOneRow func(*sql.Rows) error) error { - return simpleQueryWithArgs(conn, handleOneRow, sql) +func simpleQuery(conn *sql.Conn, query string, handleOneRow func(*sql.Rows) error) error { + return simpleQueryWithArgs(context.Background(), conn, handleOneRow, query) } -func simpleQueryWithArgs(conn *sql.Conn, handleOneRow func(*sql.Rows) error, sql string, args ...interface{}) error { - rows, err := conn.QueryContext(context.Background(), sql, args...) +func simpleQueryWithArgs(ctx context.Context, conn *sql.Conn, handleOneRow func(*sql.Rows) error, query string, args ...interface{}) error { + var ( + rows *sql.Rows + err error + ) + if len(args) > 0 { + rows, err = conn.QueryContext(ctx, query, args...) + } else { + rows, err = conn.QueryContext(ctx, query) + } if err != nil { - return errors.Annotatef(err, "sql: %s, args: %s", sql, args) + return errors.Annotatef(err, "sql: %s, args: %s", query, args) } defer rows.Close() for rows.Next() { if err := handleOneRow(rows); err != nil { rows.Close() - return errors.Annotatef(err, "sql: %s, args: %s", sql, args) + return errors.Annotatef(err, "sql: %s, args: %s", query, args) } } - rows.Close() - return errors.Annotatef(rows.Err(), "sql: %s, args: %s", sql, args) + return errors.Annotatef(rows.Err(), "sql: %s, args: %s", query, args) } -func pickupPossibleField(meta TableMeta, db *sql.Conn) (string, error) { +func pickupPossibleField(tctx *tcontext.Context, meta TableMeta, db *BaseConn) (string, error) { // try using _tidb_rowid first if meta.HasImplicitRowID() { return "_tidb_rowid", nil } // try to use pk or uk - fieldName, err := getNumericIndex(db, meta) + fieldName, err := getNumericIndex(tctx, db, meta) if err != nil { return "", err } @@ -1088,7 +1106,7 @@ func pickupPossibleField(meta TableMeta, db *sql.Conn) (string, error) { return fieldName, nil } -func estimateCount(tctx *tcontext.Context, dbName, tableName string, db *sql.Conn, field string, conf *Config) uint64 { +func estimateCount(tctx *tcontext.Context, dbName, tableName string, db *BaseConn, field string, conf *Config) uint64 { var query string if strings.TrimSpace(field) == "*" || strings.TrimSpace(field) == "" { query = fmt.Sprintf("EXPLAIN SELECT * FROM `%s`.`%s`", escapeString(dbName), escapeString(tableName)) @@ -1130,44 +1148,39 @@ func estimateCount(tctx *tcontext.Context, dbName, tableName string, db *sql.Con return 0 } -func detectEstimateRows(tctx *tcontext.Context, db *sql.Conn, query string, fieldNames []string) uint64 { - rows, err := db.QueryContext(tctx, query) - if err != nil { - tctx.L().Info("can't estimate rows from db", - zap.String("query", query), log.ShortError(err)) - return 0 - } - defer rows.Close() - rows.Next() - columns, err := rows.Columns() - if err != nil { - tctx.L().Info("can't get columns when estimate rows from db", - zap.String("query", query), log.ShortError(err)) - return 0 - } - err = rows.Err() - if err != nil { - tctx.L().Info("rows meet some error during the query when estimate rows", - zap.String("query", query), log.ShortError(err)) - return 0 - } - addr := make([]interface{}, len(columns)) - oneRow := make([]sql.NullString, len(columns)) - fieldIndex := -1 - for i := range oneRow { - addr[i] = &oneRow[i] - } -found: - for i := range oneRow { - for _, fieldName := range fieldNames { - if strings.EqualFold(columns[i], fieldName) { - fieldIndex = i - break found +func detectEstimateRows(tctx *tcontext.Context, db *BaseConn, query string, fieldNames []string) uint64 { + var ( + fieldIndex int + oneRow []sql.NullString + ) + err := db.QuerySQL(tctx, func(rows *sql.Rows) error { + columns, err := rows.Columns() + if err != nil { + return errors.Trace(err) + } + addr := make([]interface{}, len(columns)) + oneRow = make([]sql.NullString, len(columns)) + fieldIndex = -1 + found: + for i := range oneRow { + for _, fieldName := range fieldNames { + if strings.EqualFold(columns[i], fieldName) { + fieldIndex = i + break found + } } } - } - err = rows.Scan(addr...) - if err != nil || fieldIndex < 0 { + if fieldIndex == -1 { + rows.Close() + return nil + } + + for i := range oneRow { + addr[i] = &oneRow[i] + } + return rows.Scan(addr...) + }, func() {}, query) + if err != nil || fieldIndex == -1 { tctx.L().Info("can't estimate rows from db", zap.String("query", query), zap.Int("fieldIndex", fieldIndex), log.ShortError(err)) return 0 @@ -1176,10 +1189,9 @@ found: estRows, err := strconv.ParseFloat(oneRow[fieldIndex].String, 64) if err != nil { tctx.L().Info("can't get parse estimate rows from db", - zap.String("query", query), log.ShortError(err)) + zap.String("query", query), zap.String("estRows", oneRow[fieldIndex].String), log.ShortError(err)) return 0 } - return uint64(estRows) } @@ -1231,10 +1243,10 @@ func escapeString(s string) string { } // GetPartitionNames get partition names from a specified table -func GetPartitionNames(db *sql.Conn, schema, table string) (partitions []string, err error) { +func GetPartitionNames(tctx *tcontext.Context, db *BaseConn, schema, table string) (partitions []string, err error) { partitions = make([]string, 0) var partitionName sql.NullString - err = simpleQueryWithArgs(db, func(rows *sql.Rows) error { + err = db.QuerySQL(tctx, func(rows *sql.Rows) error { err := rows.Scan(&partitionName) if err != nil { return errors.Trace(err) @@ -1243,6 +1255,8 @@ func GetPartitionNames(db *sql.Conn, schema, table string) (partitions []string, partitions = append(partitions, partitionName.String) } return nil + }, func() { + partitions = partitions[:0] }, "SELECT PARTITION_NAME from INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ?", schema, table) return } diff --git a/dumpling/export/sql_test.go b/dumpling/export/sql_test.go index 038edd5601abf..f52b3d882c155 100644 --- a/dumpling/export/sql_test.go +++ b/dumpling/export/sql_test.go @@ -64,15 +64,17 @@ func TestBuildSelectAllQuery(t *testing.T) { // Test TiDB server. mockConf.ServerInfo.ServerType = version.ServerTypeTiDB + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) - orderByClause, err := buildOrderByClause(mockConf, conn, database, table, true) + orderByClause, err := buildOrderByClause(tctx, mockConf, baseConn, database, table, true) require.NoError(t, err) mock.ExpectQuery("SHOW COLUMNS FROM"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err := buildSelectField(conn, database, table, false) + selectedField, _, err := buildSelectField(tctx, baseConn, database, table, false) require.NoError(t, err) q := buildSelectQuery(database, table, selectedField, "", "", orderByClause) @@ -82,14 +84,14 @@ func TestBuildSelectAllQuery(t *testing.T) { WillReturnRows(sqlmock.NewRows(showIndexHeaders). AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", "")) - orderByClause, err = buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err) mock.ExpectQuery("SHOW COLUMNS FROM"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err = buildSelectField(conn, database, table, false) + selectedField, _, err = buildSelectField(tctx, baseConn, database, table, false) require.NoError(t, err) q = buildSelectQuery(database, table, selectedField, "", "", orderByClause) @@ -107,14 +109,14 @@ func TestBuildSelectAllQuery(t *testing.T) { mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows(showIndexHeaders). AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", "")) - orderByClause, err := buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err := buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err, comment) mock.ExpectQuery("SHOW COLUMNS FROM"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err = buildSelectField(conn, database, table, false) + selectedField, _, err = buildSelectField(tctx, baseConn, database, table, false) require.NoError(t, err, comment) q = buildSelectQuery(database, table, selectedField, "", "", orderByClause) @@ -133,14 +135,14 @@ func TestBuildSelectAllQuery(t *testing.T) { mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows(showIndexHeaders)) - orderByClause, err := buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err := buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err, comment) mock.ExpectQuery("SHOW COLUMNS FROM"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err = buildSelectField(conn, "test", "t", false) + selectedField, _, err = buildSelectField(tctx, baseConn, "test", "t", false) require.NoError(t, err, comment) q := buildSelectQuery(database, table, selectedField, "", "", orderByClause) @@ -161,7 +163,7 @@ func TestBuildSelectAllQuery(t *testing.T) { WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err := buildSelectField(conn, "test", "t", false) + selectedField, _, err := buildSelectField(tctx, baseConn, "test", "t", false) require.NoError(t, err, comment) q := buildSelectQuery(database, table, selectedField, "", "", "") @@ -179,6 +181,8 @@ func TestBuildOrderByClause(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) mockConf := defaultConfigForTest(t) mockConf.SortByPk = true @@ -186,21 +190,21 @@ func TestBuildOrderByClause(t *testing.T) { // Test TiDB server. mockConf.ServerInfo.ServerType = version.ServerTypeTiDB - orderByClause, err := buildOrderByClause(mockConf, conn, database, table, true) + orderByClause, err := buildOrderByClause(tctx, mockConf, baseConn, database, table, true) require.NoError(t, err) require.Equal(t, orderByTiDBRowID, orderByClause) mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows(showIndexHeaders).AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", "")) - orderByClause, err = buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err) require.Equal(t, "ORDER BY `id`", orderByClause) // Test table with primary key. mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows(showIndexHeaders).AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", "")) - orderByClause, err = buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err) require.Equal(t, "ORDER BY `id`", orderByClause) @@ -209,7 +213,7 @@ func TestBuildOrderByClause(t *testing.T) { WillReturnRows(sqlmock.NewRows(showIndexHeaders). AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", ""). AddRow(table, 0, "PRIMARY", 2, "name", "A", 0, nil, nil, "", "BTREE", "", "")) - orderByClause, err = buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err) require.Equal(t, "ORDER BY `id`,`name`", orderByClause) @@ -217,7 +221,7 @@ func TestBuildOrderByClause(t *testing.T) { mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)). WillReturnRows(sqlmock.NewRows(showIndexHeaders)) - orderByClause, err = buildOrderByClause(mockConf, conn, database, table, false) + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) require.NoError(t, err) require.Equal(t, "", orderByClause) @@ -226,10 +230,24 @@ func TestBuildOrderByClause(t *testing.T) { for _, hasImplicitRowID := range []bool{false, true} { comment := fmt.Sprintf("current hasImplicitRowID: %v", hasImplicitRowID) - orderByClause, err := buildOrderByClause(mockConf, conn, database, table, hasImplicitRowID) + orderByClause, err := buildOrderByClause(tctx, mockConf, baseConn, database, table, hasImplicitRowID) require.NoError(t, err, comment) require.Equal(t, "", orderByClause, comment) } + + // Test build OrderByClause with retry + baseConn = newBaseConn(conn, true, func(conn *sql.Conn, b bool) (*sql.Conn, error) { + return conn, nil + }) + query := fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table) + mock.ExpectQuery(query).WillReturnError(errors.New("invalid connection")) + mock.ExpectQuery(query).WillReturnError(errors.New("invalid connection")) + mock.ExpectQuery(query).WillReturnRows(sqlmock.NewRows(showIndexHeaders).AddRow(table, 0, "PRIMARY", 1, "id", "A", 0, nil, nil, "", "BTREE", "", "")) + mockConf.SortByPk = true + orderByClause, err = buildOrderByClause(tctx, mockConf, baseConn, database, table, false) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) + require.Equal(t, "ORDER BY `id`", orderByClause) } func TestBuildSelectField(t *testing.T) { @@ -241,13 +259,15 @@ func TestBuildSelectField(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) // generate columns not found mock.ExpectQuery("SHOW COLUMNS FROM"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). AddRow("id", "int(11)", "NO", "PRI", nil, "")) - selectedField, _, err := buildSelectField(conn, "test", "t", false) + selectedField, _, err := buildSelectField(tctx, baseConn, "test", "t", false) require.Equal(t, "*", selectedField) require.NoError(t, err) require.NoError(t, mock.ExpectationsWereMet()) @@ -259,7 +279,7 @@ func TestBuildSelectField(t *testing.T) { AddRow("name", "varchar(12)", "NO", "", nil, ""). AddRow("quo`te", "varchar(12)", "NO", "UNI", nil, "")) - selectedField, _, err = buildSelectField(conn, "test", "t", true) + selectedField, _, err = buildSelectField(tctx, baseConn, "test", "t", true) require.Equal(t, "`id`,`name`,`quo``te`", selectedField) require.NoError(t, err) require.NoError(t, mock.ExpectationsWereMet()) @@ -272,10 +292,25 @@ func TestBuildSelectField(t *testing.T) { AddRow("quo`te", "varchar(12)", "NO", "UNI", nil, ""). AddRow("generated", "varchar(12)", "NO", "", nil, "VIRTUAL GENERATED")) - selectedField, _, err = buildSelectField(conn, "test", "t", false) + selectedField, _, err = buildSelectField(tctx, baseConn, "test", "t", false) require.Equal(t, "`id`,`name`,`quo``te`", selectedField) require.NoError(t, err) require.NoError(t, mock.ExpectationsWereMet()) + + // Test build SelectField with retry + baseConn = newBaseConn(conn, true, func(conn *sql.Conn, b bool) (*sql.Conn, error) { + return conn, nil + }) + mock.ExpectQuery("SHOW COLUMNS FROM").WillReturnError(errors.New("invalid connection")) + mock.ExpectQuery("SHOW COLUMNS FROM").WillReturnError(errors.New("invalid connection")) + mock.ExpectQuery("SHOW COLUMNS FROM"). + WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). + AddRow("id", "int(11)", "NO", "PRI", nil, "")) + + selectedField, _, err = buildSelectField(tctx, baseConn, "test", "t", false) + require.Equal(t, "*", selectedField) + require.NoError(t, err) + require.NoError(t, mock.ExpectationsWereMet()) } func TestParseSnapshotToTSO(t *testing.T) { @@ -315,6 +350,8 @@ func TestShowCreateView(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) mock.ExpectQuery("SHOW FIELDS FROM `test`.`v`"). WillReturnRows(sqlmock.NewRows([]string{"Field", "Type", "Null", "Key", "Default", "Extra"}). @@ -324,7 +361,7 @@ func TestShowCreateView(t *testing.T) { WillReturnRows(sqlmock.NewRows([]string{"View", "Create View", "character_set_client", "collation_connection"}). AddRow("v", "CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v` (`a`) AS SELECT `t`.`a` AS `a` FROM `test`.`t`", "utf8", "utf8_general_ci")) - createTableSQL, createViewSQL, err := ShowCreateView(conn, "test", "v") + createTableSQL, createViewSQL, err := ShowCreateView(tctx, baseConn, "test", "v") require.NoError(t, err) require.Equal(t, "CREATE TABLE `v`(\n`a` int\n)ENGINE=MyISAM;\n", createTableSQL) require.Equal(t, "DROP TABLE IF EXISTS `v`;\nDROP VIEW IF EXISTS `v`;\nSET @PREV_CHARACTER_SET_CLIENT=@@CHARACTER_SET_CLIENT;\nSET @PREV_CHARACTER_SET_RESULTS=@@CHARACTER_SET_RESULTS;\nSET @PREV_COLLATION_CONNECTION=@@COLLATION_CONNECTION;\nSET character_set_client = utf8;\nSET character_set_results = utf8;\nSET collation_connection = utf8_general_ci;\nCREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v` (`a`) AS SELECT `t`.`a` AS `a` FROM `test`.`t`;\nSET character_set_client = @PREV_CHARACTER_SET_CLIENT;\nSET character_set_results = @PREV_CHARACTER_SET_RESULTS;\nSET collation_connection = @PREV_COLLATION_CONNECTION;\n", createViewSQL) @@ -338,14 +375,16 @@ func TestShowCreatePolicy(t *testing.T) { require.NoError(t, db.Close()) }() + tctx := tcontext.Background().WithLogger(appLogger) conn, err := db.Conn(context.Background()) require.NoError(t, err) + baseConn := newBaseConn(conn, true, nil) mock.ExpectQuery("SHOW CREATE PLACEMENT POLICY `policy_x`"). WillReturnRows(sqlmock.NewRows([]string{"Policy", "Create Policy"}). AddRow("policy_x", "CREATE PLACEMENT POLICY `policy_x` LEARNERS=1")) - createPolicySQL, err := ShowCreatePlacementPolicy(conn, "policy_x") + createPolicySQL, err := ShowCreatePlacementPolicy(tctx, baseConn, "policy_x") require.NoError(t, err) require.Equal(t, "CREATE PLACEMENT POLICY `policy_x` LEARNERS=1", createPolicySQL) require.NoError(t, mock.ExpectationsWereMet()) @@ -359,13 +398,15 @@ func TestListPolicyNames(t *testing.T) { require.NoError(t, db.Close()) }() + tctx := tcontext.Background().WithLogger(appLogger) conn, err := db.Conn(context.Background()) + baseConn := newBaseConn(conn, true, nil) require.NoError(t, err) mock.ExpectQuery("select distinct policy_name from information_schema.placement_policies where policy_name is not null;"). WillReturnRows(sqlmock.NewRows([]string{"policy_name"}). AddRow("policy_x")) - policies, err := ListAllPlacementPolicyNames(conn) + policies, err := ListAllPlacementPolicyNames(tctx, baseConn) require.NoError(t, err) require.Equal(t, []string{"policy_x"}, policies) require.NoError(t, mock.ExpectationsWereMet()) @@ -374,7 +415,7 @@ func TestListPolicyNames(t *testing.T) { expectedErr := &mysql.MySQLError{Number: ErrNoSuchTable, Message: "Table 'information_schema.placement_policies' doesn't exist"} mock.ExpectExec("select distinct policy_name from information_schema.placement_policies where policy_name is not null;"). WillReturnError(expectedErr) - _, err = ListAllPlacementPolicyNames(conn) + _, err = ListAllPlacementPolicyNames(tctx, baseConn) if mysqlErr, ok := err.(*mysql.MySQLError); ok { require.Equal(t, mysqlErr.Number, ErrNoSuchTable) } @@ -417,20 +458,22 @@ func TestSelectTiDBRowID(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) database, table := "test", "t" // _tidb_rowid is unavailable, or PKIsHandle. mock.ExpectExec("SELECT _tidb_rowid from `test`.`t`"). WillReturnError(errors.New(`1054, "Unknown column '_tidb_rowid' in 'field list'"`)) - hasImplicitRowID, err := SelectTiDBRowID(conn, database, table) + hasImplicitRowID, err := SelectTiDBRowID(tctx, baseConn, database, table) require.NoError(t, err) require.False(t, hasImplicitRowID) // _tidb_rowid is available. mock.ExpectExec("SELECT _tidb_rowid from `test`.`t`"). WillReturnResult(sqlmock.NewResult(0, 0)) - hasImplicitRowID, err = SelectTiDBRowID(conn, database, table) + hasImplicitRowID, err = SelectTiDBRowID(tctx, baseConn, database, table) require.NoError(t, err) require.True(t, hasImplicitRowID) @@ -438,7 +481,7 @@ func TestSelectTiDBRowID(t *testing.T) { expectedErr := errors.New("mock error") mock.ExpectExec("SELECT _tidb_rowid from `test`.`t`"). WillReturnError(expectedErr) - hasImplicitRowID, err = SelectTiDBRowID(conn, database, table) + hasImplicitRowID, err = SelectTiDBRowID(tctx, baseConn, database, table) require.ErrorIs(t, errors.Cause(err), expectedErr) require.False(t, hasImplicitRowID) } @@ -452,7 +495,7 @@ func TestBuildTableSampleQueries(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) - + baseConn := newBaseConn(conn, true, nil) tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel() d := &Dumper{ @@ -730,7 +773,7 @@ func TestBuildTableSampleQueries(t *testing.T) { } } - require.NoError(t, d.concurrentDumpTable(tctx, conn, meta, taskChan)) + require.NoError(t, d.concurrentDumpTable(tctx, baseConn, meta, taskChan)) require.NoError(t, mock.ExpectationsWereMet()) orderByClause := buildOrderByClauseString(handleColNames) @@ -862,7 +905,7 @@ func TestBuildRegionQueriesWithoutPartition(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) - + baseConn := newBaseConn(conn, true, nil) tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel() d := &Dumper{ @@ -996,7 +1039,7 @@ func TestBuildRegionQueriesWithoutPartition(t *testing.T) { mock.ExpectQuery(fmt.Sprintf("SHOW INDEX FROM `%s`.`%s`", database, table)).WillReturnRows(rows) mock.ExpectQuery("SHOW INDEX FROM").WillReturnRows(sqlmock.NewRows(showIndexHeaders)) } - require.NoError(t, d.concurrentDumpTable(tctx, conn, meta, taskChan)) + require.NoError(t, d.concurrentDumpTable(tctx, baseConn, meta, taskChan)) require.NoError(t, mock.ExpectationsWereMet()) for i, w := range testCase.expectedWhereClauses { @@ -1021,7 +1064,7 @@ func TestBuildRegionQueriesWithPartitions(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) - + baseConn := newBaseConn(conn, true, nil) tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel() d := &Dumper{ @@ -1194,7 +1237,7 @@ func TestBuildRegionQueriesWithPartitions(t *testing.T) { } orderByClause := buildOrderByClauseString(handleColNames) - require.NoError(t, d.concurrentDumpTable(tctx, conn, meta, taskChan)) + require.NoError(t, d.concurrentDumpTable(tctx, baseConn, meta, taskChan)) require.NoError(t, mock.ExpectationsWereMet()) chunkIdx := 0 @@ -1256,7 +1299,7 @@ func TestBuildVersion3RegionQueries(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) - + baseConn := newBaseConn(conn, true, nil) tctx, cancel := tcontext.Background().WithLogger(appLogger).WithCancel() oldOpenFunc := openDBFunc defer func() { @@ -1502,7 +1545,7 @@ func TestBuildVersion3RegionQueries(t *testing.T) { } orderByClause := buildOrderByClauseString(handleColNames) - require.NoError(t, d.concurrentDumpTable(tctx, conn, meta, taskChan)) + require.NoError(t, d.concurrentDumpTable(tctx, baseConn, meta, taskChan)) require.NoError(t, mock.ExpectationsWereMet()) chunkIdx := 0 @@ -1578,6 +1621,8 @@ func TestPickupPossibleField(t *testing.T) { conn, err := db.Conn(context.Background()) require.NoError(t, err) + tctx := tcontext.Background().WithLogger(appLogger) + baseConn := newBaseConn(conn, true, nil) meta := &mockTableIR{ dbName: database, @@ -1701,7 +1746,7 @@ func TestPickupPossibleField(t *testing.T) { mock.ExpectQuery(query).WillReturnRows(rows) } - field, err := pickupPossibleField(meta, conn) + field, err := pickupPossibleField(tctx, meta, baseConn) if expectedErr != nil { require.ErrorIs(t, err, expectedErr) } else { diff --git a/dumpling/export/sql_type.go b/dumpling/export/sql_type.go index 511fd43870750..d12a8e61757c4 100644 --- a/dumpling/export/sql_type.go +++ b/dumpling/export/sql_type.go @@ -11,10 +11,9 @@ import ( var colTypeRowReceiverMap = map[string]func() RowReceiverStringer{} var ( - nullValue = "NULL" - quotationMark = []byte{'\''} - twoQuotationMarks = []byte{'\'', '\''} - doubleQuotationMark = []byte{'"'} + nullValue = "NULL" + quotationMark = []byte{'\''} + twoQuotationMarks = []byte{'\'', '\''} ) // There are two kinds of scenes to use this dataType diff --git a/dumpling/export/util.go b/dumpling/export/util.go index 2b4875679df97..cd91008139284 100644 --- a/dumpling/export/util.go +++ b/dumpling/export/util.go @@ -10,7 +10,7 @@ import ( "time" "github.com/pingcap/errors" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tidb/br/pkg/version" tcontext "github.com/pingcap/tidb/dumpling/context" diff --git a/dumpling/export/test_util.go b/dumpling/export/util_for_test.go similarity index 100% rename from dumpling/export/test_util.go rename to dumpling/export/util_for_test.go diff --git a/dumpling/export/writer.go b/dumpling/export/writer.go index ca6bfdad35f77..ca7dd438ad756 100644 --- a/dumpling/export/writer.go +++ b/dumpling/export/writer.go @@ -29,7 +29,7 @@ type Writer struct { receivedTaskCount int - rebuildConnFn func(*sql.Conn) (*sql.Conn, error) + rebuildConnFn func(*sql.Conn, bool) (*sql.Conn, error) finishTaskCallBack func(Task) finishTableCallBack func(Task) } @@ -181,7 +181,7 @@ func (w *Writer) WriteTableData(meta TableMeta, ir TableDataIR, currentChunk int zap.String("table", meta.TableName()), zap.Int("chunkIndex", currentChunk), zap.NamedError("lastError", lastErr)) // don't rebuild connection when dump for the first time if retryTime > 1 { - conn, err = w.rebuildConnFn(conn) + conn, err = w.rebuildConnFn(conn, true) w.conn = conn if err != nil { return @@ -199,7 +199,7 @@ func (w *Writer) WriteTableData(meta TableMeta, ir TableDataIR, currentChunk int } defer ir.Close() return w.tryToWriteTableData(tctx, meta, ir, currentChunk) - }, newDumpChunkBackoffer(canRebuildConn(conf.Consistency, conf.TransactionalConsistency))) + }, newRebuildConnBackOffer(canRebuildConn(conf.Consistency, conf.TransactionalConsistency))) } func (w *Writer) tryToWriteTableData(tctx *tcontext.Context, meta TableMeta, ir TableDataIR, curChkIdx int) error { diff --git a/dumpling/export/writer_serial_test.go b/dumpling/export/writer_serial_test.go index a606784a1bd6b..95b406d51dd21 100644 --- a/dumpling/export/writer_serial_test.go +++ b/dumpling/export/writer_serial_test.go @@ -121,7 +121,7 @@ func TestWriteInsertInCsv(t *testing.T) { bf := storage.NewBufferWriter() // test nullValue - opt := &csvOption{separator: []byte(","), delimiter: doubleQuotationMark, nullValue: "\\N"} + opt := &csvOption{separator: []byte(","), delimiter: []byte{'"'}, nullValue: "\\N"} conf := configForWriteCSV(cfg, true, opt) n, err := WriteInsertInCsv(tcontext.Background(), conf, tableIR, tableIR, bf) require.Equal(t, uint64(4), n) @@ -217,7 +217,7 @@ func TestWriteInsertInCsvReturnsError(t *testing.T) { bf := storage.NewBufferWriter() // test nullValue - opt := &csvOption{separator: []byte(","), delimiter: doubleQuotationMark, nullValue: "\\N"} + opt := &csvOption{separator: []byte(","), delimiter: []byte{'"'}, nullValue: "\\N"} conf := configForWriteCSV(cfg, true, opt) n, err := WriteInsertInCsv(tcontext.Background(), conf, tableIR, tableIR, bf) require.Equal(t, uint64(3), n) diff --git a/dumpling/export/writer_util.go b/dumpling/export/writer_util.go index 2ccbefad372f9..b673243c6a8d8 100755 --- a/dumpling/export/writer_util.go +++ b/dumpling/export/writer_util.go @@ -226,7 +226,7 @@ func WriteInsert(pCtx *tcontext.Context, cfg *Config, meta TableMeta, tblIR Tabl } counter++ wp.AddFileSize(uint64(bf.Len()-lastBfSize) + 2) // 2 is for ",\n" and ";\n" - failpoint.Inject("ChaosBrokenMySQLConn", func(_ failpoint.Value) { + failpoint.Inject("ChaosBrokenWriterConn", func(_ failpoint.Value) { failpoint.Return(0, errors.New("connection is closed")) }) diff --git a/dumpling/tests/chaos/run.sh b/dumpling/tests/chaos/run.sh index d7c0b7d955446..12b4ceba8bd8d 100755 --- a/dumpling/tests/chaos/run.sh +++ b/dumpling/tests/chaos/run.sh @@ -20,7 +20,7 @@ run_sql "insert into $DB_NAME.$TABLE_NAME values $(seq -s, 100 | sed 's/,*$//g' # dumping with consistency none export DUMPLING_TEST_DATABASE=$DB_NAME -export GO_FAILPOINTS="github.com/pingcap/tidb/dumpling/export/ChaosBrokenMySQLConn=1*return" +export GO_FAILPOINTS="github.com/pingcap/tidb/dumpling/export/ChaosBrokenWriterConn=1*return;github.com/pingcap/tidb/dumpling/export/ChaosBrokenMetaConn=1*return" run_dumpling --consistency=none --loglevel debug # check data record count diff --git a/errno/errcode.go b/errno/errcode.go index 046a7614bdb32..e926b9ffe53df 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1007,11 +1007,15 @@ const ( ErrMultiStatementDisabled = 8130 ErrPartitionStatsMissing = 8131 ErrNotSupportedWithSem = 8132 - ErrDataInConsistentExtraIndex = 8133 - ErrDataInConsistentMisMatchIndex = 8134 + ErrDataInconsistentMismatchCount = 8133 + ErrDataInconsistentMismatchIndex = 8134 ErrAsOf = 8135 ErrVariableNoLongerSupported = 8136 ErrAnalyzeMissColumn = 8137 + ErrInconsistentRowValue = 8138 + ErrInconsistentHandle = 8139 + ErrInconsistentIndexedValue = 8140 + ErrAssertionFailed = 8141 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 @@ -1037,7 +1041,7 @@ const ( ErrWriteOnSnapshot = 8220 ErrInvalidKey = 8221 ErrInvalidIndexKey = 8222 - ErrDataInConsistent = 8223 + ErrDataInconsistent = 8223 ErrDDLJobNotFound = 8224 ErrCancelFinishedDDLJob = 8225 ErrCannotCancelDDLJob = 8226 diff --git a/errno/errname.go b/errno/errname.go index 1fd5b54744645..ab44d3bd18860 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1000,7 +1000,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), ErrInvalidKey: mysql.Message("invalid key", nil), ErrInvalidIndexKey: mysql.Message("invalid index key", nil), - ErrDataInConsistent: mysql.Message("index:%#v != record:%#v", []int{0, 1}), + ErrDataInconsistent: mysql.Message("data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v", []int{2, 3, 4}), ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), @@ -1008,8 +1008,12 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), - ErrDataInConsistentExtraIndex: mysql.Message("handle %#v, index:%#v != record:%#v", []int{0, 1, 2}), - ErrDataInConsistentMisMatchIndex: mysql.Message("col %s, handle %#v, index:%#v != record:%#v, compare err:%#v", []int{1, 2, 3, 4}), + ErrDataInconsistentMismatchCount: mysql.Message("data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d", nil), + ErrDataInconsistentMismatchIndex: mysql.Message("data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v", []int{3, 4, 5, 6}), + ErrInconsistentRowValue: mysql.Message("writing inconsistent data in table: %s, expected-values:{%s} != record-values:{%s}", []int{1, 2}), + ErrInconsistentHandle: mysql.Message("writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v", []int{2, 3, 4, 5}), + ErrInconsistentIndexedValue: mysql.Message("writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s}", []int{3, 4}), + ErrAssertionFailed: mysql.Message("assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v", []int{0}), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index b1a29debfc300..46e05aaefd1c6 100644 --- a/errors.toml +++ b/errors.toml @@ -233,7 +233,7 @@ TiDB admin check table failed. ["admin:8223"] error = ''' -index:%#v != record:%#v +data inconsistency in table: %s, index: %s, handle: %s, index-values:%#v != record-values:%#v ''' ["admin:8224"] @@ -506,6 +506,16 @@ error = ''' This partition function is not allowed ''' +["ddl:1567"] +error = ''' +Incorrect partition name +''' + +["ddl:1652"] +error = ''' +Duplicate partition field name '%-.192s' +''' + ["ddl:1654"] error = ''' Partition column values of incorrect type @@ -918,12 +928,12 @@ Export failed: %s ["executor:8133"] error = ''' -handle %#v, index:%#v != record:%#v +data inconsistency in table: %s, index: %s, index-count:%d != record-count:%d ''' ["executor:8134"] error = ''' -col %s, handle %#v, index:%#v != record:%#v, compare err:%#v +data inconsistency in table: %s, index: %s, col: %s, handle: %#v, index-values:%#v != record-values:%#v, compare err:%#v ''' ["executor:8212"] @@ -1281,6 +1291,11 @@ error = ''' View '%-.192s.%-.192s' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them ''' +["planner:1391"] +error = ''' +Key part '%-.192s' length cannot be 0 +''' + ["planner:1462"] error = ''' `%-.192s`.`%-.192s` contains view recursion @@ -1811,6 +1826,21 @@ error = ''' column %s can't be in none state ''' +["table:8138"] +error = ''' +writing inconsistent data in table: %s, expected-values:{%s} != record-values:{%s} +''' + +["table:8139"] +error = ''' +writing inconsistent data in table: %s, index: %s, index-handle:%#v != record-handle:%#v, index: %#v, record: %#v +''' + +["table:8140"] +error = ''' +writing inconsistent data in table: %s, index: %s, col: %s, indexed-value:{%s} != record-value:{%s} +''' + ["tikv:1105"] error = ''' Unknown error @@ -1831,6 +1861,11 @@ error = ''' Statement aborted because lock(s) could not be acquired immediately and NOWAIT is set. ''' +["tikv:8141"] +error = ''' +assertion failed: key: %s, assertion: %s, start_ts: %v, existing start ts: %v, existing commit ts: %v +''' + ["tikv:8229"] error = ''' TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction diff --git a/executor/adapter.go b/executor/adapter.go index fa96d82ea5e84..368d31e18560a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -677,7 +677,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { keys = filterTemporaryTableKeys(sctx.GetSessionVars(), keys) seVars := sctx.GetSessionVars() keys = filterLockTableKeys(seVars.StmtCtx, keys) - lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout) + lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout, len(keys)) var lockKeyStats *util.LockKeysDetails ctx = context.WithValue(ctx, util.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() @@ -966,6 +966,10 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo sessVars.DurationParse = 0 // Clean the stale read flag when statement execution finish sessVars.StmtCtx.IsStaleness = false + + if sessVars.StmtCtx.ReadFromTableCache { + metrics.ReadFromTableCacheCounter.Inc() + } } // CloseRecordSet will finish the execution of current statement and do some record work diff --git a/executor/admin.go b/executor/admin.go index fec0fec14ca94..600fb48f6b970 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -467,7 +467,7 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa return result, err } - _, err = e.index.Create(e.ctx, txn, row.idxVals, row.handle, row.rsData) + _, err = e.index.Create(e.ctx, txn, row.idxVals, row.handle, row.rsData, table.WithIgnoreAssertion) if err != nil { return result, err } diff --git a/executor/admin_test.go b/executor/admin_test.go index 72aa436babe9b..8a7496f53ef02 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -17,22 +17,33 @@ package executor_test import ( "context" "fmt" + "os" + "strings" "testing" "time" "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" mysql "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" ) func TestAdminCheckIndexRange(t *testing.T) { @@ -227,7 +238,7 @@ func TestAdminRecoverIndex(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.True(t, executor.ErrAdminCheckTable.Equal(err)) + require.True(t, consistency.ErrAdminCheckInconsistent.Equal(err)) err = tk.ExecToErr("admin check index admin_test c2") require.Error(t, err) @@ -329,7 +340,7 @@ func TestClusteredIndexAdminRecoverIndex(t *testing.T) { require.NoError(t, err) err = txn.Commit(context.Background()) require.NoError(t, err) - tk.MustGetErrCode("admin check table t", mysql.ErrAdminCheckTable) + tk.MustGetErrCode("admin check table t", mysql.ErrDataInconsistent) tk.MustGetErrCode("admin check index t idx", mysql.ErrAdminCheckTable) tk.MustQuery("SELECT COUNT(*) FROM t USE INDEX(idx)").Check(testkit.Rows("2")) @@ -368,7 +379,7 @@ func TestAdminRecoverPartitionTableIndex(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.True(t, executor.ErrAdminCheckTable.Equal(err)) + require.True(t, consistency.ErrAdminCheckInconsistent.Equal(err)) r := tk.MustQuery("SELECT COUNT(*) FROM admin_test USE INDEX(c2)") r.Check(testkit.Rows("2")) @@ -866,8 +877,8 @@ func TestAdminCheckPartitionTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") require.Error(t, err) - require.EqualError(t, err, fmt.Sprintf("[executor:8003]admin_test_p err:[admin:8223]index: != record:&admin.RecordData{Handle:%d, Values:[]types.Datum{types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}}}", i, i)) - require.True(t, executor.ErrAdminCheckTable.Equal(err)) + require.EqualError(t, err, fmt.Sprintf("[admin:8223]data inconsistency in table: admin_test_p, index: idx, handle: %d, index-values:\"\" != record-values:\"handle: %d, values: [KindInt64 %d]\"", i, i, i)) + require.True(t, consistency.ErrAdminCheckInconsistent.Equal(err)) // TODO: fix admin recover for partition table. // r := tk.MustQuery("admin recover index admin_test_p idx") // r.Check(testkit.Rows("0 0")) @@ -894,7 +905,8 @@ func TestAdminCheckPartitionTableFailed(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") - require.EqualError(t, err, fmt.Sprintf("[executor:8133]handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:", i+8, i+8)) + require.Error(t, err) + require.EqualError(t, err, fmt.Sprintf("[admin:8223]data inconsistency in table: admin_test_p, index: idx, handle: %d, index-values:\"handle: %d, values: [KindInt64 %d KindInt64 %d]\" != record-values:\"\"", i+8, i+8, i+8, i+8)) // TODO: fix admin recover for partition table. txn, err = store.Begin() require.NoError(t, err) @@ -916,7 +928,8 @@ func TestAdminCheckPartitionTableFailed(t *testing.T) { err = txn.Commit(context.Background()) require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test_p") - require.EqualError(t, err, fmt.Sprintf("[executor:8134]col c2, handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:", i, i+8, i)) + require.Error(t, err) + require.EqualError(t, err, fmt.Sprintf("[executor:8134]data inconsistency in table: admin_test_p, index: idx, col: c2, handle: \"%d\", index-values:\"KindInt64 %d\" != record-values:\"KindInt64 %d\", compare err:", i, i+8, i)) // TODO: fix admin recover for partition table. txn, err = store.Begin() require.NoError(t, err) @@ -928,11 +941,381 @@ func TestAdminCheckPartitionTableFailed(t *testing.T) { } } +const dbName, tblName = "test", "admin_test" + +type inconsistencyTestKit struct { + *testkit.AsyncTestKit + uniqueIndex table.Index + plainIndex table.Index + ctx context.Context + sctx *stmtctx.StatementContext + t *testing.T +} + +type kitOpt struct { + pkColType string + idxColType string + ukColType string + clustered string +} + +func newDefaultOpt() *kitOpt { + return &kitOpt{ + pkColType: "int", + idxColType: "int", + ukColType: "varchar(255)", + } +} + +func newInconsistencyKit(t *testing.T, tk *testkit.AsyncTestKit, opt *kitOpt) *inconsistencyTestKit { + ctx := tk.OpenSession(context.Background(), dbName) + se := testkit.TryRetrieveSession(ctx) + i := &inconsistencyTestKit{ + AsyncTestKit: tk, + ctx: ctx, + sctx: se.GetSessionVars().StmtCtx, + t: t, + } + tk.MustExec(i.ctx, "drop table if exists "+tblName) + tk.MustExec(i.ctx, + fmt.Sprintf("create table %s (c1 %s, c2 %s, c3 %s, primary key(c1) %s, index uk1(c2), index k2(c3))", + tblName, opt.pkColType, opt.idxColType, opt.ukColType, opt.clustered), + ) + i.rebuild() + return i +} + +func (tk *inconsistencyTestKit) rebuild() { + tk.MustExec(tk.ctx, "truncate table "+tblName) + is := domain.GetDomain(testkit.TryRetrieveSession(tk.ctx)).InfoSchema() + tbl, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + require.NoError(tk.t, err) + tk.uniqueIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[0]) + tk.plainIndex = tables.NewIndex(tbl.Meta().ID, tbl.Meta(), tbl.Meta().Indices[1]) +} + +type logEntry struct { + entry zapcore.Entry + fields []zapcore.Field +} + +func (l *logEntry) checkMsg(t *testing.T, msg string) { + require.Equal(t, msg, l.entry.Message) +} + +func (l *logEntry) checkField(t *testing.T, requireFields ...zapcore.Field) { + for _, rf := range requireFields { + var f *zapcore.Field + for i, field := range l.fields { + if field.Equals(rf) { + f = &l.fields[i] + break + } + } + require.NotNilf(t, f, "matched log fields %s:%s not found in log", rf.Key, rf) + } + +} + +func (l *logEntry) checkFieldNotEmpty(t *testing.T, fieldName string) { + var f *zapcore.Field + for i, field := range l.fields { + if field.Key == fieldName { + f = &l.fields[i] + break + } + } + require.NotNilf(t, f, "log field %s not found in log", fieldName) + require.NotEmpty(t, f.String) +} + +type logHook struct { + zapcore.Core + logs []logEntry + enc zapcore.Encoder + messageFilter string +} + +func (h *logHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { + h.logs = append(h.logs, logEntry{entry: entry, fields: fields}) + return nil +} + +func (h *logHook) Check(entry zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { + if len(h.messageFilter) > 0 && !strings.Contains(entry.Message, h.messageFilter) { + return nil + } + return ce.AddCore(entry, h) +} + +func (h *logHook) encode(entry *logEntry) (string, error) { + buffer, err := h.enc.EncodeEntry(entry.entry, entry.fields) + if err != nil { + return "", err + } + return buffer.String(), nil +} + +func (h *logHook) checkLogCount(t *testing.T, expected int) { + logsStr := make([]string, len(h.logs)) + for i, item := range h.logs { + var err error + logsStr[i], err = h.encode(&item) + require.NoError(t, err) + } + // Check the length of strings, so that in case the test fails, the error message will be printed. + require.Len(t, logsStr, expected) +} + +func withLogHook(ctx context.Context, msgFilter string) (newCtx context.Context, hook *logHook) { + conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} + _, r, _ := log.InitLogger(conf) + enc := log.NewTextEncoder(&config.GetGlobalConfig().Log.ToLogConfig().Config) + hook = &logHook{r.Core, nil, enc, msgFilter} + logger := zap.New(hook) + newCtx = context.WithValue(ctx, logutil.CtxLogKey, logger) + return +} + +func TestCheckFailReport(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := newInconsistencyKit(t, testkit.NewAsyncTestKit(t, store), newDefaultOpt()) + + // row more than unique index + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 1, '10')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.uniqueIndex.Delete(tk.sctx, txn, types.MakeDatums(1), kv.IntHandle(1))) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindInt64 1]\"", err.Error()) + hook.checkLogCount(t, 1) + hook.logs[0].checkMsg(t, "admin check found data inconsistency") + hook.logs[0].checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "uk1"), + zap.Stringer("row_id", kv.IntHandle(1)), + ) + hook.logs[0].checkFieldNotEmpty(t, "row_mvcc") + }() + + // row more than plain index + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 1, '10')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.plainIndex.Delete(tk.sctx, txn, []types.Datum{types.NewStringDatum("10")}, kv.IntHandle(1))) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindString 10]\"", err.Error()) + hook.checkLogCount(t, 1) + hook.logs[0].checkMsg(t, "admin check found data inconsistency") + hook.logs[0].checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "k2"), + zap.Stringer("row_id", kv.IntHandle(1)), + ) + hook.logs[0].checkFieldNotEmpty(t, "row_mvcc") + }() + + // row is missed for plain key + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + _, err = tk.plainIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewStringDatum("100")}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"handle: 1, values: [KindString 100 KindInt64 1]\" != record-values:\"\"", err.Error()) + hook.checkLogCount(t, 1) + logEntry := hook.logs[0] + logEntry.checkMsg(t, "admin check found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "k2"), + zap.Stringer("row_id", kv.IntHandle(1)), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc") + logEntry.checkFieldNotEmpty(t, "index_mvcc") + + // test inconsistency check in index lookup + ctx, hook = withLogHook(tk.ctx, "") + rs, err := tk.Exec(ctx, "select * from admin_test use index(k2) where c3 = '100'") + require.NoError(t, err) + _, err = session.GetRows4Test(ctx, testkit.TryRetrieveSession(ctx), rs) + require.Error(t, err) + require.Equal(t, "[executor:8133]data inconsistency in table: admin_test, index: k2, index-count:1 != record-count:0", err.Error()) + hook.checkLogCount(t, 1) + logEntry = hook.logs[0] + logEntry.checkMsg(t, "indexLookup found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "k2"), + zap.Int64("table_cnt", 0), + zap.Int64("index_cnt", 1), + zap.String("missing_handles", `[1]`), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc_0") + }() + + // row is missed for unique key + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewIntDatum(10)}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"handle: 1, values: [KindInt64 10 KindInt64 1]\" != record-values:\"\"", err.Error()) + hook.checkLogCount(t, 1) + logEntry := hook.logs[0] + logEntry.checkMsg(t, "admin check found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "uk1"), + zap.Stringer("row_id", kv.IntHandle(1)), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc") + logEntry.checkFieldNotEmpty(t, "index_mvcc") + + // test inconsistency check in point-get + ctx, hook = withLogHook(tk.ctx, "") + rs, err := tk.Exec(ctx, "select * from admin_test use index(uk1) where c2 = 10") + require.NoError(t, err) + _, err = session.GetRows4Test(ctx, testkit.TryRetrieveSession(ctx), rs) + require.Error(t, err) + hook.checkLogCount(t, 1) + logEntry = hook.logs[0] + logEntry.checkMsg(t, "indexLookup found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "uk1"), + zap.Int64("table_cnt", 0), + zap.Int64("index_cnt", 1), + zap.String("missing_handles", `[1]`), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc_0") + }() + + // handle match but value is different for uk + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 10, '100')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.uniqueIndex.Delete(tk.sctx, txn, []types.Datum{types.NewIntDatum(10)}, kv.IntHandle(1))) + _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewIntDatum(20)}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: uk1, col: c2, handle: \"1\", index-values:\"KindInt64 20\" != record-values:\"KindInt64 10\", compare err:", err.Error()) + hook.checkLogCount(t, 1) + logEntry := hook.logs[0] + logEntry.checkMsg(t, "admin check found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "uk1"), + zap.Stringer("row_id", kv.IntHandle(1)), + zap.String("col", "c2"), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc") + logEntry.checkFieldNotEmpty(t, "index_mvcc") + }() + + // handle match but value is different for plain key + func() { + defer tk.rebuild() + + tk.MustExec(tk.ctx, fmt.Sprintf("insert into %s values(1, 10, '100')", tblName)) + txn, err := store.Begin() + require.NoError(t, err) + require.NoError(t, tk.plainIndex.Delete(tk.sctx, txn, []types.Datum{types.NewStringDatum("100")}, kv.IntHandle(1))) + _, err = tk.plainIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewStringDatum("200")}, kv.IntHandle(1), nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: k2, col: c3, handle: \"1\", index-values:\"KindString 200\" != record-values:\"KindString 100\", compare err:", err.Error()) + hook.checkLogCount(t, 1) + logEntry := hook.logs[0] + logEntry.checkMsg(t, "admin check found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "k2"), + zap.Stringer("row_id", kv.IntHandle(1)), + zap.String("col", "c3"), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc") + logEntry.checkFieldNotEmpty(t, "index_mvcc") + }() + + // test binary column. + opt := newDefaultOpt() + opt.clustered = "clustered" + opt.pkColType = "varbinary(300)" + opt.idxColType = "varbinary(300)" + opt.ukColType = "varbinary(300)" + tk = newInconsistencyKit(t, testkit.NewAsyncTestKit(t, store), newDefaultOpt()) + func() { + defer tk.rebuild() + + txn, err := store.Begin() + require.NoError(t, err) + encoded, err := codec.EncodeKey(new(stmtctx.StatementContext), nil, types.NewBytesDatum([]byte{1, 0, 1, 0, 0, 1, 1})) + require.Nil(t, err) + hd, err := kv.NewCommonHandle(encoded) + require.NoError(t, err) + _, err = tk.uniqueIndex.Create(mock.NewContext(), txn, []types.Datum{types.NewBytesDatum([]byte{1, 1, 0, 1, 1, 1, 1, 0})}, hd, nil) + require.NoError(t, err) + require.NoError(t, txn.Commit(tk.ctx)) + + ctx, hook := withLogHook(tk.ctx, "inconsistency") + _, err = tk.Exec(ctx, "admin check table admin_test") + require.Error(t, err) + require.Equal(t, `[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 282574488403969, index-values:"handle: 282574488403969, values: [KindInt64 282578800083201 KindInt64 282574488403969]" != record-values:""`, err.Error()) + hook.checkLogCount(t, 1) + logEntry := hook.logs[0] + logEntry.checkMsg(t, "admin check found data inconsistency") + logEntry.checkField(t, + zap.String("table_name", "admin_test"), + zap.String("index_name", "uk1"), + zap.Stringer("row_id", kv.IntHandle(282574488403969)), + ) + logEntry.checkFieldNotEmpty(t, "row_mvcc") + logEntry.checkFieldNotEmpty(t, "index_mvcc") + }() +} + func TestAdminCheckTable(t *testing.T) { // test NULL value. store, clean := testkit.CreateMockStore(t) defer clean() - tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`CREATE TABLE test_null ( @@ -1157,12 +1540,12 @@ func TestAdminCheckTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8003]admin_test err:[admin:8223]index: != record:&admin.RecordData{Handle:-1, Values:[]types.Datum{types.Datum{k:0x1, decimal:0x0, length:0x0, i:-10, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}}}") - require.True(t, executor.ErrAdminCheckTable.Equal(err)) + require.EqualError(t, err, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: -1, index-values:\"\" != record-values:\"handle: -1, values: [KindInt64 -10]\"") + require.True(t, consistency.ErrAdminCheckInconsistent.Equal(err)) tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8003]admin_test err:[admin:8223]index:\"?\" != record:\"?\"") + require.EqualError(t, err, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: ?, index-values:\"?\" != record-values:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") r := tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 7")) @@ -1179,11 +1562,11 @@ func TestAdminCheckTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8133]handle 0, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:0, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + require.EqualError(t, err, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: 0, index-values:\"handle: 0, values: [KindInt64 0 KindInt64 0]\" != record-values:\"\"") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8133]handle \"?\", index:\"?\" != record:\"?\"") + require.EqualError(t, err, "[admin:8223]data inconsistency in table: admin_test, index: c2, handle: ?, index-values:\"?\" != record-values:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Add one row of index. @@ -1202,11 +1585,11 @@ func TestAdminCheckTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle 2, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:13, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:12, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"2\", index-values:\"KindInt64 13\" != record-values:\"KindInt64 12\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. @@ -1221,11 +1604,11 @@ func TestAdminCheckTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"10\", index-values:\"KindInt64 19\" != record-values:\"KindInt64 20\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. @@ -1240,11 +1623,11 @@ func TestAdminCheckTableFailed(t *testing.T) { require.NoError(t, err) err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"10\", index-values:\"KindInt64 19\" != record-values:\"KindInt64 20\", compare err:") tk.MustExec("set @@tidb_redact_log=1;") err = tk.ExecToErr("admin check table admin_test") require.Error(t, err) - require.EqualError(t, err, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + require.EqualError(t, err, "[executor:8134]data inconsistency in table: admin_test, index: c2, col: c2, handle: \"?\", index-values:\"?\" != record-values:\"?\", compare err:\"?\"") tk.MustExec("set @@tidb_redact_log=0;") // Recover records. diff --git a/executor/aggfuncs/main_test.go b/executor/aggfuncs/main_test.go index a385a98c57215..5ff0437f1e59f 100644 --- a/executor/aggfuncs/main_test.go +++ b/executor/aggfuncs/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/executor/analyze.go b/executor/analyze.go index 134b4ba303c04..9ac68e9223021 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -259,11 +259,7 @@ func (e *AnalyzeExec) saveAnalyzeOptsV2() error { idx += 1 } exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, sql.String()) - if err != nil { - return err - } - _, _, err = exec.ExecRestrictedStmt(context.TODO(), stmt) + _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, sql.String()) if err != nil { return err } @@ -1656,13 +1652,7 @@ type AnalyzeFastExec struct { func (e *AnalyzeFastExec) calculateEstimateSampleStep() (err error) { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - var stmt ast.StmtNode - stmt, err = exec.ParseWithParams(context.TODO(), true, "select flag from mysql.stats_histograms where table_id = %?", e.tableID.GetStatisticsID()) - if err != nil { - return - } - var rows []chunk.Row - rows, _, err = exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "select flag from mysql.stats_histograms where table_id = %?", e.tableID.GetStatisticsID()) if err != nil { return } diff --git a/executor/analyze_test.go b/executor/analyze_test.go index c167eff12bcf3..0be814d76a068 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -182,12 +182,10 @@ func TestAnalyzeRestrict(t *testing.T) { } func TestAnalyzeParameters(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - tk := testkit.NewTestKit(t, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -243,12 +241,10 @@ func TestAnalyzeParameters(t *testing.T) { } func TestAnalyzeTooLongColumns(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - tk := testkit.NewTestKit(t, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a json)") @@ -669,10 +665,9 @@ func TestFailedAnalyzeRequest(t *testing.T) { } func TestExtractTopN(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_extract_topn") tk.MustExec("use test_extract_topn") @@ -724,12 +719,10 @@ func TestExtractTopN(t *testing.T) { } func TestHashInTopN(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - tk := testkit.NewTestKit(t, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b float, c decimal(30, 10), d varchar(20))") @@ -818,11 +811,11 @@ func TestNormalAnalyzeOnCommonHandle(t *testing.T) { } func TestDefaultValForAnalyze(t *testing.T) { - t.Skip("skip race test") store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) - + tk.MustExec("set @@tidb_analyze_version=1") + defer tk.MustExec("set @@tidb_analyze_version=2") tk.MustExec("drop database if exists test_default_val_for_analyze;") tk.MustExec("create database test_default_val_for_analyze;") tk.MustExec("use test_default_val_for_analyze") @@ -837,10 +830,10 @@ func TestDefaultValForAnalyze(t *testing.T) { tk.MustQuery("select @@tidb_enable_fast_analyze").Check(testkit.Rows("0")) tk.MustQuery("select @@session.tidb_enable_fast_analyze").Check(testkit.Rows("0")) tk.MustExec("analyze table t with 0 topn;") - tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows("IndexReader_6 512.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 512.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) - tk.MustQuery("explain format = 'brief' select * from t where a = 999").Check(testkit.Rows("IndexReader_6 0.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.00 cop[tikv] table:t, index:a(a) range:[999,999], keep order:false")) + tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows("IndexReader 512.00 root index:IndexRangeScan", + "└─IndexRangeScan 512.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) + tk.MustQuery("explain format = 'brief' select * from t where a = 999").Check(testkit.Rows("IndexReader 0.00 root index:IndexRangeScan", + "└─IndexRangeScan 0.00 cop[tikv] table:t, index:a(a) range:[999,999], keep order:false")) tk.MustExec("drop table t;") tk.MustExec("create table t (a int, key(a));") @@ -851,8 +844,8 @@ func TestDefaultValForAnalyze(t *testing.T) { tk.MustExec("insert into t values (?)", i) } tk.MustExec("analyze table t with 0 topn;") - tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows("IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) + tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows("IndexReader 1.00 root index:IndexRangeScan", + "└─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false")) } func TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColumn(t *testing.T) { @@ -1115,8 +1108,6 @@ func testAnalyzeIncremental(tk *testkit.TestKit, t *testing.T, dom *domain.Domai } func TestIssue20874(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1283,7 +1274,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { tk.MustQuery("select * from t where b > 1 and c > 1") require.NoError(t, h.LoadNeededHistograms()) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) + require.NoError(t, err) tableInfo := table.Meta() tbl := h.GetTableStats(tableInfo) lastVersion := tbl.Version @@ -1312,7 +1303,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { require.Equal(t, 2, len(col0.Buckets)) // manual analyze uses the table-level persisted options by merging the new options - tk.MustExec("analyze table t columns a,b with 0.9 samplerate, 3 buckets") + tk.MustExec("analyze table t columns a,b with 1 samplerate, 3 buckets") tbl = h.GetTableStats(tableInfo) require.Greater(t, tbl.Version, lastVersion) lastVersion = tbl.Version @@ -1327,7 +1318,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { // The columns are: table_id, sample_num, sample_rate, buckets, topn, column_choice, column_ids. rs = tk.MustQuery("select * from mysql.analyze_options where table_id=" + strconv.FormatInt(tbl.PhysicalID, 10)) require.Equal(t, 1, len(rs.Rows())) - require.Equal(t, "0.9", rs.Rows()[0][2]) + require.Equal(t, "1", rs.Rows()[0][2]) require.Equal(t, "3", rs.Rows()[0][3]) require.Equal(t, "1", rs.Rows()[0][4]) require.Equal(t, "LIST", rs.Rows()[0][5]) @@ -1381,7 +1372,7 @@ PARTITION BY RANGE ( a ) ( tk.MustQuery("select * from t where b > 1 and c > 1") require.NoError(t, h.LoadNeededHistograms()) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) + require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() require.NotNil(t, pi) @@ -1497,7 +1488,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec("analyze table t partition p2") is = dom.InfoSchema() table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) + require.NoError(t, err) tableInfo = table.Meta() pi = tableInfo.GetPartitionInfo() p2 := h.GetPartitionStats(tableInfo, pi.Definitions[2].ID) @@ -1590,7 +1581,7 @@ PARTITION BY RANGE ( a ) ( }() is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) + require.NoError(t, err) tableInfo := table.Meta() pi := tableInfo.GetPartitionInfo() require.NotNil(t, pi) @@ -1709,9 +1700,9 @@ func TestSavedAnalyzeOptionsForMultipleTables(t *testing.T) { tk.MustExec("analyze table t1,t2 with 2 topn") is := dom.InfoSchema() table1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - require.Nil(t, err) + require.NoError(t, err) table2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) - require.Nil(t, err) + require.NoError(t, err) tableInfo1 := table1.Meta() tableInfo2 := table2.Meta() tblStats1 := h.GetTableStats(tableInfo1) @@ -1769,7 +1760,7 @@ func TestSavedAnalyzeColumnOptions(t *testing.T) { }() is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.Nil(t, err) + require.NoError(t, err) tblInfo := tbl.Meta() tk.MustExec("select * from t where b > 1") require.NoError(t, h.DumpColStatsUsageToKV()) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index f4a8d822116b6..871d6452257f9 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" @@ -477,8 +478,22 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { if len(val) == 0 { if e.idxInfo != nil && (!e.tblInfo.IsCommonHandle || !e.idxInfo.Primary) && !e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", - e.idxInfo.Name.O, e.handles[i]) + return (&consistency.Reporter{ + HandleEncode: func(_ kv.Handle) kv.Key { + return key + }, + IndexEncode: func(_ *consistency.RecordData) kv.Key { + return indexKeys[i] + }, + Tbl: e.tblInfo, + Idx: e.idxInfo, + Sctx: e.ctx, + }).ReportLookupInconsistent(ctx, + 1, 0, + e.handles[i:i+1], + e.handles, + []consistency.RecordData{{}}, + ) } continue } @@ -525,7 +540,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { // LockKeys locks the keys for pessimistic transaction. func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, keys ...kv.Key) error { txnCtx := seCtx.GetSessionVars().TxnCtx - lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime) + lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime, len(keys)) if txnCtx.IsPessimistic { lctx.InitReturnValues(len(keys)) } diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 3dbe99ad96554..a36ad9eba5f39 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/benchdaily" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/memory" @@ -1303,7 +1302,7 @@ func (tc indexJoinTestCase) getMockDataSourceOptByRows(rows int) mockDataSourceP } } -func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { +func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) (Executor, error) { outerCols, innerCols := tc.columns(), tc.columns() joinSchema := expression.NewSchema(outerCols...) joinSchema.Append(innerCols...) @@ -1317,6 +1316,13 @@ func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, for i := range keyOff2IdxOff { keyOff2IdxOff[i] = i } + + readerBuilder, err := newExecutorBuilder(tc.ctx, nil, nil, 0, false, oracle.GlobalTxnScope). + newDataReaderBuilder(&mockPhysicalIndexReader{e: innerDS}) + if err != nil { + return nil, err + } + e := &IndexLookUpJoin{ baseExecutor: newBaseExecutor(tc.ctx, joinSchema, 1, outerDS), outerCtx: outerCtx{ @@ -1325,7 +1331,7 @@ func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, hashCols: tc.outerHashKeyIdx, }, innerCtx: innerCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil, nil, 0, false, oracle.GlobalTxnScope)}, + readerBuilder: readerBuilder, rowTypes: rightTypes, colLens: colLens, keyCols: tc.innerJoinKeyIdx, @@ -1338,21 +1344,24 @@ func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, lastColHelper: nil, } e.joinResult = newFirstChunk(e) - return e + return e, nil } -func prepare4IndexOuterHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { - e := prepare4IndexInnerHashJoin(tc, outerDS, innerDS).(*IndexLookUpJoin) - idxHash := &IndexNestedLoopHashJoin{IndexLookUpJoin: *e} +func prepare4IndexOuterHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) (Executor, error) { + e, err := prepare4IndexInnerHashJoin(tc, outerDS, innerDS) + if err != nil { + return nil, err + } + idxHash := &IndexNestedLoopHashJoin{IndexLookUpJoin: *e.(*IndexLookUpJoin)} concurrency := tc.concurrency idxHash.joiners = make([]joiner, concurrency) for i := 0; i < concurrency; i++ { - idxHash.joiners[i] = e.joiner.Clone() + idxHash.joiners[i] = e.(*IndexLookUpJoin).joiner.Clone() } - return idxHash + return idxHash, nil } -func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) Executor { +func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, innerDS *mockDataSource) (Executor, error) { outerCols, innerCols := tc.columns(), tc.columns() joinSchema := expression.NewSchema(outerCols...) joinSchema.Append(innerCols...) @@ -1381,6 +1390,13 @@ func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, inne compareFuncs = append(compareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], innerJoinKeys[i])) outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(nil, outerJoinKeys[i], outerJoinKeys[i])) } + + readerBuilder, err := newExecutorBuilder(tc.ctx, nil, nil, 0, false, oracle.GlobalTxnScope). + newDataReaderBuilder(&mockPhysicalIndexReader{e: innerDS}) + if err != nil { + return nil, err + } + e := &IndexLookUpMergeJoin{ baseExecutor: newBaseExecutor(tc.ctx, joinSchema, 2, outerDS), outerMergeCtx: outerMergeCtx{ @@ -1391,7 +1407,7 @@ func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, inne compareFuncs: outerCompareFuncs, }, innerMergeCtx: innerMergeCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil, nil, 0, false, oracle.GlobalTxnScope)}, + readerBuilder: readerBuilder, rowTypes: rightTypes, joinKeys: innerJoinKeys, colLens: colLens, @@ -1409,7 +1425,7 @@ func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, inne joiners[i] = newJoiner(tc.ctx, 0, false, defaultValues, nil, leftTypes, rightTypes, nil) } e.joiners = joiners - return e + return e, nil } type indexJoinType int8 @@ -1431,13 +1447,18 @@ func benchmarkIndexJoinExecWithCase( for i := 0; i < b.N; i++ { b.StopTimer() var exec Executor + var err error switch execType { case indexInnerHashJoin: - exec = prepare4IndexInnerHashJoin(tc, outerDS, innerDS) + exec, err = prepare4IndexInnerHashJoin(tc, outerDS, innerDS) case indexOuterHashJoin: - exec = prepare4IndexOuterHashJoin(tc, outerDS, innerDS) + exec, err = prepare4IndexOuterHashJoin(tc, outerDS, innerDS) case indexMergeJoin: - exec = prepare4IndexMergeJoin(tc, outerDS, innerDS) + exec, err = prepare4IndexMergeJoin(tc, outerDS, innerDS) + } + + if err != nil { + b.Fatal(err) } tmpCtx := context.Background() @@ -1446,7 +1467,7 @@ func benchmarkIndexJoinExecWithCase( innerDS.prepareChunks() b.StartTimer() - if err := exec.Open(tmpCtx); err != nil { + if err = exec.Open(tmpCtx); err != nil { b.Fatal(err) } for { @@ -2101,9 +2122,3 @@ func BenchmarkPipelinedRowNumberWindowFunctionExecution(b *testing.B) { b.ReportAllocs() } - -func TestBenchDaily(t *testing.T) { - benchdaily.Run( - BenchmarkReadLastLinesOfHugeLine, - ) -} diff --git a/executor/brie.go b/executor/brie.go index 49bb57cd98fab..3920254e2b869 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -462,11 +462,7 @@ func (gs *tidbGlueSession) CreateSession(store kv.Storage) (glue.Session, error) // These queries execute without privilege checking, since the calling statements // such as BACKUP and RESTORE have already been privilege checked. func (gs *tidbGlueSession) Execute(ctx context.Context, sql string) error { - stmt, err := gs.se.(sqlexec.RestrictedSQLExecutor).ParseWithParams(ctx, true, sql) - if err != nil { - return err - } - _, _, err = gs.se.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(ctx, stmt) + _, _, err := gs.se.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, nil, sql) return err } diff --git a/executor/builder.go b/executor/builder.go index d1892401763b3..7ff7b3445bfff 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -86,7 +86,8 @@ var ( type executorBuilder struct { ctx sessionctx.Context is infoschema.InfoSchema - snapshotTS uint64 // The consistent snapshot timestamp for the executor to read data. + snapshotTS uint64 // The ts for snapshot-read. A select statement without for update will use this ts + forUpdateTS uint64 // The ts should be used by insert/update/delete/select-for-update statement snapshotTSCached bool err error // err is set when there is error happened during Executor building process. hasLock bool @@ -97,6 +98,14 @@ type executorBuilder struct { inUpdateStmt bool inDeleteStmt bool inInsertStmt bool + inSelectLockStmt bool + + // forDataReaderBuilder indicates whether the builder is used by a dataReaderBuilder. + // When forDataReader is true, the builder should use the dataReaderTS as the executor read ts. This is because + // dataReaderBuilder can be used in concurrent goroutines, so we must ensure that getting the ts should be thread safe and + // can return a correct value even if the session context has already been destroyed + forDataReaderBuilder bool + dataReaderTS uint64 } // CTEStorages stores resTbl and iterInTbl for CTEExec. @@ -143,6 +152,7 @@ func (b *MockExecutorBuilder) Build(p plannercore.Plan) Executor { } func (b *executorBuilder) build(p plannercore.Plan) Executor { + var e Executor switch v := p.(type) { case nil: return nil @@ -257,13 +267,13 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { case *plannercore.Analyze: return b.buildAnalyze(v) case *plannercore.PhysicalTableReader: - return b.buildTableReader(v) + e = b.buildTableReader(v) case *plannercore.PhysicalTableSample: return b.buildTableSample(v) case *plannercore.PhysicalIndexReader: - return b.buildIndexReader(v) + e = b.buildIndexReader(v) case *plannercore.PhysicalIndexLookUpReader: - return b.buildIndexLookUpReader(v) + e = b.buildIndexLookUpReader(v) case *plannercore.PhysicalWindow: return b.buildWindow(v) case *plannercore.PhysicalShuffle: @@ -294,6 +304,57 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { b.err = ErrUnknownPlan.GenWithStack("Unknown Plan %T", p) return nil } + + if tblExec, ok := e.(dataSourceExecutor); ok { + tbl := tblExec.Table() + tableInfo := tbl.Meta() + // When reading from a cached table, check whether it satisfies the conditions of read cache. + if tableInfo.TableCacheStatusType == model.TableCacheStatusEnable { + physicalPlan := p.(plannercore.PhysicalPlan) + return b.buildCachedTableExecutor(tbl, physicalPlan, e) + } + } + + return e +} + +// buildCachedTableExecutor adds an UnionScan to the original Executor to make the reader read from table cache. +func (b *executorBuilder) buildCachedTableExecutor(tbl table.Table, p plannercore.PhysicalPlan, e Executor) Executor { + if b.ctx.GetSessionVars().SnapshotTS != 0 || b.ctx.GetSessionVars().StmtCtx.IsStaleness { + return e + } + + cachedTable := tbl.(table.CachedTable) + startTS, err := b.getSnapshotTS() + if err != nil { + b.err = errors.Trace(err) + return nil + } + + leaseDuration := time.Duration(variable.TableCacheLease.Load()) * time.Second + sessionVars := b.ctx.GetSessionVars() + // Use the TS of the transaction to determine whether the cache can be used. + cacheData := cachedTable.TryReadFromCache(startTS, leaseDuration) + if cacheData != nil { + sessionVars.StmtCtx.ReadFromTableCache = true + switch raw := e.(type) { + case *TableReaderExecutor: + raw.dummy = true + case *IndexReaderExecutor: + raw.dummy = true + case *IndexLookUpExecutor: + raw.dummy = true + } + us := plannercore.PhysicalUnionScan{CacheTable: cacheData}.Init(b.ctx, nil, -1) + us.SetChildren(p) + e = b.buildUnionScanFromReader(e, us) + } else { + if !b.inUpdateStmt && !b.inDeleteStmt && !b.inInsertStmt && !sessionVars.StmtCtx.InExplainStmt { + store := b.ctx.GetStore() + cachedTable.UpdateLockForRead(context.Background(), store, startTS, leaseDuration) + } + } + return e } func (b *executorBuilder) buildCancelDDLJobs(v *plannercore.CancelDDLJobs) Executor { @@ -361,10 +422,13 @@ func (b *executorBuilder) buildShowDDL(v *plannercore.ShowDDL) Executor { } func (b *executorBuilder) buildShowDDLJobs(v *plannercore.PhysicalShowDDLJobs) Executor { + loc := b.ctx.GetSessionVars().Location() + ddlJobRetriever := DDLJobRetriever{TZLoc: loc} e := &ShowDDLJobsExec{ - jobNumber: int(v.JobNumber), - is: b.is, - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + jobNumber: int(v.JobNumber), + is: b.is, + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + DDLJobRetriever: ddlJobRetriever, } return e } @@ -630,12 +694,16 @@ func (b *executorBuilder) buildDeallocate(v *plannercore.Deallocate) Executor { } func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor { + if !b.inSelectLockStmt { + b.inSelectLockStmt = true + defer func() { b.inSelectLockStmt = false }() + } b.hasLock = true if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { return nil } // Build 'select for update' using the 'for update' ts. - b.snapshotTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() src := b.build(v.Children()[0]) if b.err != nil { @@ -822,7 +890,7 @@ func (b *executorBuilder) buildInsert(v *plannercore.Insert) Executor { return nil } } - b.snapshotTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selectExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -1476,8 +1544,24 @@ func (b *executorBuilder) buildTableDual(v *plannercore.PhysicalTableDual) Execu return e } -// `getSnapshotTS` returns the timestamp of the snapshot that a reader should read. +// `getSnapshotTS` returns for-update-ts if in insert/update/delete/lock statement otherwise the isolation read ts +// Please notice that in RC isolation, the above two ts are the same func (b *executorBuilder) getSnapshotTS() (uint64, error) { + if b.forDataReaderBuilder { + return b.dataReaderTS, nil + } + + if (b.inInsertStmt || b.inUpdateStmt || b.inDeleteStmt || b.inSelectLockStmt) && b.forUpdateTS != 0 { + return b.forUpdateTS, nil + } + + return b.getReadTS() +} + +// getReadTS returns the ts used by select (without for-update clause). The return value is affected by the isolation level +// and some stale/historical read contexts. For example, it will return txn.StartTS in RR and return +// the current timestamp in RC isolation +func (b *executorBuilder) getReadTS() (uint64, error) { // `refreshForUpdateTSForRC` should always be invoked before returning the cached value to // ensure the correct value is returned even the `snapshotTS` field is already set by other // logics. However for `IndexLookUpMergeJoin` and `IndexLookUpHashJoin`, it requires caching the @@ -1626,6 +1710,14 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo timeRange: v.QueryTimeRange, }, } + case strings.ToLower(infoschema.TableTiKVRegionPeers): + return &MemTableReaderExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + table: v.Table, + retriever: &tikvRegionPeersRetriever{ + extractor: v.Extractor.(*plannercore.TikvRegionPeersExtractor), + }, + } case strings.ToLower(infoschema.TableSchemata), strings.ToLower(infoschema.TableStatistics), strings.ToLower(infoschema.TableTiDBIndexes), @@ -1647,7 +1739,6 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableProcesslist), strings.ToLower(infoschema.ClusterTableProcesslist), strings.ToLower(infoschema.TableTiKVRegionStatus), - strings.ToLower(infoschema.TableTiKVRegionPeers), strings.ToLower(infoschema.TableTiDBHotRegions), strings.ToLower(infoschema.TableSessionVar), strings.ToLower(infoschema.TableConstraints), @@ -1743,9 +1834,12 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo }, } case strings.ToLower(infoschema.TableDDLJobs): + loc := b.ctx.GetSessionVars().Location() + ddlJobRetriever := DDLJobRetriever{TZLoc: loc} return &DDLJobsReaderExec{ - baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), - is: b.is, + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + is: b.is, + DDLJobRetriever: ddlJobRetriever, } case strings.ToLower(infoschema.TableTiFlashTables), strings.ToLower(infoschema.TableTiFlashSegments): @@ -1903,17 +1997,6 @@ func (b *executorBuilder) buildMaxOneRow(v *plannercore.PhysicalMaxOneRow) Execu } func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executor { - // A quick fix for avoiding a race mentioned in issue #30468. - // Fetch the snapshot ts to make the transaction's state ready. Otherwise, multiple threads in the Union executor - // may change the transaction's state concurrently, which causes race. - // This fix is a hack, but with minimal change to the current code and works. Actually, the usage of the transaction - // states and the logic to access the snapshot ts should all be refactored. - _, err := b.getSnapshotTS() - if err != nil { - b.err = err - return nil - } - childExecs := make([]Executor, len(v.Children())) for i, child := range v.Children() { childExecs[i] = b.build(child) @@ -2011,7 +2094,7 @@ func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { return nil } - b.snapshotTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2068,7 +2151,7 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { if b.err = b.updateForUpdateTSIfNeeded(v.SelectPlan); b.err != nil { return nil } - b.snapshotTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() + b.forUpdateTS = b.ctx.GetSessionVars().TxnCtx.GetForUpdateTS() selExec := b.build(v.SelectPlan) if b.err != nil { return nil @@ -2407,11 +2490,7 @@ func (b *executorBuilder) getApproximateTableCountFromStorage(sctx sessionctx.Co if task.PartitionName != "" { sqlexec.MustFormatSQL(sql, " partition(%n)", task.PartitionName) } - stmt, err := b.ctx.(sqlexec.RestrictedSQLExecutor).ParseWithParams(context.TODO(), true, sql.String()) - if err != nil { - return 0, false - } - rows, _, err := b.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := b.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(context.TODO(), nil, sql.String()) if err != nil { return 0, false } @@ -2778,6 +2857,22 @@ func (b *executorBuilder) corColInAccess(p plannercore.PhysicalPlan) bool { return false } +func (b *executorBuilder) newDataReaderBuilder(p plannercore.PhysicalPlan) (*dataReaderBuilder, error) { + ts, err := b.getSnapshotTS() + if err != nil { + return nil, err + } + + builderForDataReader := *b + builderForDataReader.forDataReaderBuilder = true + builderForDataReader.dataReaderTS = ts + + return &dataReaderBuilder{ + Plan: p, + executorBuilder: &builderForDataReader, + }, nil +} + func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) Executor { outerExec := b.build(v.Children()[1-v.InnerChildIdx]) if b.err != nil { @@ -2849,6 +2944,13 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) break } } + + readerBuilder, err := b.newDataReaderBuilder(innerPlan) + if err != nil { + b.err = err + return nil + } + e := &IndexLookUpJoin{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), outerExec), outerCtx: outerCtx{ @@ -2857,7 +2959,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) filter: outerFilter, }, innerCtx: innerCtx{ - readerBuilder: &dataReaderBuilder{Plan: innerPlan, executorBuilder: b}, + readerBuilder: readerBuilder, rowTypes: innerTypes, hashTypes: innerHashTypes, colLens: v.IdxColLens, @@ -2959,6 +3061,12 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex } executorCounterIndexLookUpJoin.Inc() + readerBuilder, err := b.newDataReaderBuilder(innerPlan) + if err != nil { + b.err = err + return nil + } + e := &IndexLookUpMergeJoin{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID(), outerExec), outerMergeCtx: outerMergeCtx{ @@ -2970,7 +3078,7 @@ func (b *executorBuilder) buildIndexLookUpMergeJoin(v *plannercore.PhysicalIndex compareFuncs: v.OuterCompareFuncs, }, innerMergeCtx: innerMergeCtx{ - readerBuilder: &dataReaderBuilder{Plan: innerPlan, executorBuilder: b}, + readerBuilder: readerBuilder, rowTypes: innerTypes, joinKeys: v.InnerJoinKeys, keyCols: innerKeyCols, @@ -3151,6 +3259,10 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E return nil } + if ret.table.Meta().TempTableType != model.TempTableNone { + ret.dummy = true + } + ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) @@ -3378,6 +3490,10 @@ func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) E return nil } + if ret.table.Meta().TempTableType != model.TempTableNone { + ret.dummy = true + } + ret.ranges = is.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexNames = append(sctx.IndexNames, is.Table.Name.O+":"+is.Index.Name.O) @@ -3473,6 +3589,12 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn if err != nil { return nil, err } + + readerBuilder, err := b.newDataReaderBuilder(nil) + if err != nil { + return nil, err + } + e := &IndexLookUpExecutor{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), dagPB: indexReq, @@ -3486,7 +3608,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn indexStreaming: indexStreaming, tableStreaming: tableStreaming, indexPaging: indexPaging, - dataReaderBuilder: &dataReaderBuilder{executorBuilder: b}, + dataReaderBuilder: readerBuilder, corColInIdxSide: b.corColInDistPlan(v.IndexPlans), corColInTblSide: b.corColInDistPlan(v.TablePlans), corColInAccess: b.corColInAccess(v.IndexPlans[0]), @@ -3543,6 +3665,10 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo return nil } + if ret.table.Meta().TempTableType != model.TempTableNone { + ret.dummy = true + } + ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = is.Ranges @@ -3628,6 +3754,12 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd if err != nil { return nil, err } + + readerBuilder, err := b.newDataReaderBuilder(nil) + if err != nil { + return nil, err + } + e := &IndexMergeReaderExecutor{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), dagPBs: partialReqs, @@ -3641,7 +3773,7 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd tableStreaming: tableStreaming, partialPlans: v.PartialPlans, tblPlans: v.TablePlans, - dataReaderBuilder: &dataReaderBuilder{executorBuilder: b}, + dataReaderBuilder: readerBuilder, feedbacks: feedbacks, handleCols: ts.HandleCols, isCorColInPartialFilters: isCorColInPartialFilters, @@ -3764,7 +3896,11 @@ func (builder *dataReaderBuilder) buildExecutorForIndexJoinInternal(ctx context. func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager, canReorderHandles bool, memTracker *memory.Tracker, interruptSignal *atomic.Value) (Executor, error) { - childBuilder := &dataReaderBuilder{Plan: v.Children()[0], executorBuilder: builder.executorBuilder} + childBuilder, err := builder.newDataReaderBuilder(v.Children()[0]) + if err != nil { + return nil, err + } + reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc, canReorderHandles, memTracker, interruptSignal) if err != nil { return nil, err @@ -3949,9 +4085,6 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T if err != nil { return nil, err } - if builder.ctx.GetSessionVars().StmtCtx.WeakConsistency { - reqBuilderWithRange.SetIsolationLevel(kv.RC) - } kvReq, err := reqBuilderWithRange. SetDAGRequest(e.dagPB). SetStartTS(startTS). diff --git a/executor/collation_test.go b/executor/collation_test.go index 83d0b7251d6bd..d4b957ee413d6 100644 --- a/executor/collation_test.go +++ b/executor/collation_test.go @@ -21,15 +21,11 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) func TestVecGroupChecker(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tp := &types.FieldType{Tp: mysql.TypeVarchar} col0 := &expression.Column{ RetType: tp, diff --git a/executor/cte_test.go b/executor/cte_test.go index bf6d33ede4a42..52c2e957a2e21 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -27,8 +27,8 @@ import ( ) func TestBasicCTE(t *testing.T) { - store, close := testkit.CreateMockStore(t) - defer close() + store, clean := testkit.CreateMockStore(t) + defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -76,8 +76,8 @@ func TestBasicCTE(t *testing.T) { } func TestUnionDistinct(t *testing.T) { - store, close := testkit.CreateMockStore(t) - defer close() + store, clean := testkit.CreateMockStore(t) + defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") @@ -103,8 +103,8 @@ func TestUnionDistinct(t *testing.T) { } func TestCTEMaxRecursionDepth(t *testing.T) { - store, close := testkit.CreateMockStore(t) - defer close() + store, clean := testkit.CreateMockStore(t) + defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") @@ -144,8 +144,8 @@ func TestCTEMaxRecursionDepth(t *testing.T) { } func TestCTEWithLimit(t *testing.T) { - store, close := testkit.CreateMockStore(t) - defer close() + store, clean := testkit.CreateMockStore(t) + defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") @@ -357,8 +357,8 @@ func TestSpillToDisk(t *testing.T) { conf.OOMUseTmpStorage = true }) - store, close := testkit.CreateMockStore(t) - defer close() + store, clean := testkit.CreateMockStore(t) + defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") diff --git a/executor/ddl.go b/executor/ddl.go index dad26efad4102..10ea571b5bf31 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -492,11 +492,7 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx zap.String("table", fullti.Name.O), ) exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) - if err != nil { - return err - } - _, _, err = exec.ExecRestrictedStmt(context.TODO(), stmt) + _, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "admin check table %n.%n", fullti.Schema.O, fullti.Name.O) if err != nil { return err } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 29f34ed4622e3..70d8ed60e41fb 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -20,9 +20,9 @@ import ( "math" "strconv" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl" ddltestutil "github.com/pingcap/tidb/ddl/testutil" @@ -38,18 +38,20 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *testSuite6) TestTruncateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTruncateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists truncate_test;`) tk.MustExec(`create table truncate_test (a int)`) @@ -65,21 +67,25 @@ func (s *testSuite6) TestTruncateTable(c *C) { // 1. Execute the SQL of "begin"; // 2. A SQL that will fail to execute; // 3. Execute DDL. -func (s *testSuite6) TestInTxnExecDDLFail(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInTxnExecDDLFail(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (i int key);") tk.MustExec("insert into t values (1);") tk.MustExec("begin;") tk.MustExec("insert into t values (1);") _, err := tk.Exec("truncate table t;") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1' for key 'PRIMARY'") result := tk.MustQuery("select count(*) from t") result.Check(testkit.Rows("1")) } -func (s *testSuite6) TestInTxnExecDDLInvalid(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInTxnExecDDLInvalid(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (c_int int, c_str varchar(40));") @@ -89,18 +95,20 @@ func (s *testSuite6) TestInTxnExecDDLInvalid(c *C) { tk.MustExec("alter table t add index idx_4 (c_str);") } -func (s *testSuite6) TestCreateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Test create an exist database _, err := tk.Exec("CREATE database test") - c.Assert(err, NotNil) + require.Error(t, err) // Test create an exist table tk.MustExec("CREATE TABLE create_test (id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") _, err = tk.Exec("CREATE TABLE create_test (id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") - c.Assert(err, NotNil) + require.Error(t, err) // Test "if not exist" tk.MustExec("CREATE TABLE if not exists test(id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") @@ -109,65 +117,65 @@ func (s *testSuite6) TestCreateTable(c *C) { tk.MustExec(`create table issue312_1 (c float(24));`) tk.MustExec(`create table issue312_2 (c float(25));`) rs, err := tk.Exec(`desc issue312_1`) - c.Assert(err, IsNil) + require.NoError(t, err) ctx := context.Background() req := rs.NewChunk(nil) it := chunk.NewIterator4Chunk(req) for { err1 := rs.Next(ctx, req) - c.Assert(err1, IsNil) + require.NoError(t, err1) if req.NumRows() == 0 { break } for row := it.Begin(); row != it.End(); row = it.Next() { - c.Assert(row.GetString(1), Equals, "float") + require.Equal(t, "float", row.GetString(1)) } } rs, err = tk.Exec(`desc issue312_2`) - c.Assert(err, IsNil) + require.NoError(t, err) req = rs.NewChunk(nil) it = chunk.NewIterator4Chunk(req) for { err1 := rs.Next(ctx, req) - c.Assert(err1, IsNil) + require.NoError(t, err1) if req.NumRows() == 0 { break } for row := it.Begin(); row != it.End(); row = it.Next() { - c.Assert(req.GetRow(0).GetString(1), Equals, "double") + require.Equal(t, "double", req.GetRow(0).GetString(1)) } } - c.Assert(rs.Close(), IsNil) + require.NoError(t, rs.Close()) // test multiple collate specified in column when create. tk.MustExec("drop table if exists test_multiple_column_collate;") tk.MustExec("create table test_multiple_column_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - t, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) - c.Assert(err, IsNil) - c.Assert(t.Cols()[0].Charset, Equals, "utf8") - c.Assert(t.Cols()[0].Collate, Equals, "utf8_general_ci") - c.Assert(t.Meta().Charset, Equals, "utf8mb4") - c.Assert(t.Meta().Collate, Equals, "utf8mb4_bin") + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + require.NoError(t, err) + require.Equal(t, "utf8", tt.Cols()[0].Charset) + require.Equal(t, "utf8_general_ci", tt.Cols()[0].Collate) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) tk.MustExec("drop table if exists test_multiple_column_collate;") tk.MustExec("create table test_multiple_column_collate (a char(1) charset utf8 collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) - c.Assert(err, IsNil) - c.Assert(t.Cols()[0].Charset, Equals, "utf8") - c.Assert(t.Cols()[0].Collate, Equals, "utf8_general_ci") - c.Assert(t.Meta().Charset, Equals, "utf8mb4") - c.Assert(t.Meta().Collate, Equals, "utf8mb4_bin") + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + require.NoError(t, err) + require.Equal(t, "utf8", tt.Cols()[0].Charset) + require.Equal(t, "utf8_general_ci", tt.Cols()[0].Collate) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) // test Err case for multiple collate specified in column when create. tk.MustExec("drop table if exists test_err_multiple_collate;") _, err = tk.Exec("create table test_err_multiple_collate (a char(1) charset utf8mb4 collate utf8_unicode_ci collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_unicode_ci", "utf8mb4").Error()) + require.Error(t, err) + require.Equal(t, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_unicode_ci", "utf8mb4").Error(), err.Error()) tk.MustExec("drop table if exists test_err_multiple_collate;") _, err = tk.Exec("create table test_err_multiple_collate (a char(1) collate utf8_unicode_ci collate utf8mb4_general_ci) charset utf8mb4 collate utf8mb4_bin") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_general_ci", "utf8").Error()) + require.Error(t, err) + require.Equal(t, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_general_ci", "utf8").Error(), err.Error()) // table option is auto-increment tk.MustExec("drop table if exists create_auto_increment_test;") @@ -198,8 +206,10 @@ func (s *testSuite6) TestCreateTable(c *C) { tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1051|Unknown table 'test.t2_if_exists'", "Note|1051|Unknown table 'test.t3_if_exists'")) } -func (s *testSuite6) TestCreateView(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // create an source table tk.MustExec("CREATE TABLE source_table (id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") @@ -207,10 +217,10 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustExec("CREATE VIEW view_t AS select id , name from source_table") defer tk.MustExec("DROP VIEW IF EXISTS view_t") _, err := tk.Exec("CREATE VIEW view_t AS select id , name from source_table") - c.Assert(err.Error(), Equals, "[schema:1050]Table 'test.view_t' already exists") + require.EqualError(t, err, "[schema:1050]Table 'test.view_t' already exists") // create view on nonexistent table _, err = tk.Exec("create view v1 (c,d) as select a,b from t1") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.t1' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.t1' doesn't exist") // simple view tk.MustExec("create table t1 (a int ,b int)") tk.MustExec("insert into t1 values (1,2), (1,3), (2,4), (2,5), (3,10)") @@ -226,26 +236,26 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustExec("create view v5 as select * from t1") tk.MustExec("create view v6 (c,d) as select * from t1") _, err = tk.Exec("create view v7 (c,d,e) as select * from t1") - c.Assert(err.Error(), Equals, ddl.ErrViewWrongList.Error()) + require.Equal(t, ddl.ErrViewWrongList.Error(), err.Error()) // drop multiple views in a statement tk.MustExec("drop view v1,v2,v3,v4,v5,v6") // view with variable tk.MustExec("create view v1 (c,d) as select a,b+@@global.max_user_connections from t1") _, err = tk.Exec("create view v1 (c,d) as select a,b from t1 where a = @@global.max_user_connections") - c.Assert(err.Error(), Equals, "[schema:1050]Table 'test.v1' already exists") + require.EqualError(t, err, "[schema:1050]Table 'test.v1' already exists") tk.MustExec("drop view v1") // view with different col counts _, err = tk.Exec("create view v1 (c,d,e) as select a,b from t1 ") - c.Assert(err.Error(), Equals, ddl.ErrViewWrongList.Error()) + require.Equal(t, ddl.ErrViewWrongList.Error(), err.Error()) _, err = tk.Exec("create view v1 (c) as select a,b from t1 ") - c.Assert(err.Error(), Equals, ddl.ErrViewWrongList.Error()) + require.Equal(t, ddl.ErrViewWrongList.Error(), err.Error()) // view with or_replace flag tk.MustExec("drop view if exists v1") tk.MustExec("create view v1 (c,d) as select a,b from t1") tk.MustExec("create or replace view v1 (c,d) as select a,b from t1 ") tk.MustExec("create table if not exists t1 (a int ,b int)") _, err = tk.Exec("create or replace view t1 as select * from t1") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "t1", "VIEW").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "t1", "VIEW").Error(), err.Error()) // create view using prepare tk.MustExec(`prepare stmt from "create view v10 (x) as select 1";`) tk.MustExec("execute stmt") @@ -254,7 +264,7 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustExec("drop table if exists t1, t2") tk.MustExec("drop view if exists v") _, err = tk.Exec("create view v as select * from t1 union select * from t2") - c.Assert(terror.ErrorEqual(err, infoschema.ErrTableNotExists), IsTrue) + require.True(t, terror.ErrorEqual(err, infoschema.ErrTableNotExists)) tk.MustExec("create table t1(a int, b int)") tk.MustExec("create table t2(a int, b int)") tk.MustExec("insert into t1 values(1,2), (1,1), (1,2)") @@ -263,13 +273,13 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 2", "1 3")) tk.MustExec("alter table t1 drop column a") _, err = tk.Exec("select * from v") - c.Assert(terror.ErrorEqual(err, plannercore.ErrViewInvalid), IsTrue) + require.True(t, terror.ErrorEqual(err, plannercore.ErrViewInvalid)) tk.MustExec("alter table t1 add column a int") tk.MustQuery("select * from v").Sort().Check(testkit.Rows("1 1", "1 3", " 1", " 2")) tk.MustExec("alter table t1 drop column a") tk.MustExec("alter table t2 drop column b") _, err = tk.Exec("select * from v") - c.Assert(terror.ErrorEqual(err, plannercore.ErrViewInvalid), IsTrue) + require.True(t, terror.ErrorEqual(err, plannercore.ErrViewInvalid)) tk.MustExec("drop view v") tk.MustExec("create view v as (select * from t1)") @@ -289,17 +299,19 @@ func (s *testSuite6) TestCreateView(c *C) { tk.MustExec("create definer='root'@'localhost' view v_nested as select * from test_v_nested") tk.MustExec("create definer='root'@'localhost' view v_nested2 as select * from v_nested") _, err = tk.Exec("create or replace definer='root'@'localhost' view v_nested as select * from v_nested2") - c.Assert(terror.ErrorEqual(err, plannercore.ErrNoSuchTable), IsTrue) + require.True(t, terror.ErrorEqual(err, plannercore.ErrNoSuchTable)) tk.MustExec("drop table test_v_nested") tk.MustExec("drop view v_nested, v_nested2") // Refer https://github.com/pingcap/tidb/issues/25876 err = tk.ExecToErr("create view v_stale as select * from source_table as of timestamp current_timestamp(3)") - c.Assert(terror.ErrorEqual(err, executor.ErrViewInvalid), IsTrue, Commentf("err %s", err)) + require.Truef(t, terror.ErrorEqual(err, executor.ErrViewInvalid), "err %s", err) } -func (s *testSuite6) TestViewRecursion(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestViewRecursion(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists t(a int)") tk.MustExec("create definer='root'@'localhost' view recursive_view1 as select * from t") @@ -307,21 +319,25 @@ func (s *testSuite6) TestViewRecursion(c *C) { tk.MustExec("drop table t") tk.MustExec("rename table recursive_view2 to t") _, err := tk.Exec("select * from recursive_view1") - c.Assert(terror.ErrorEqual(err, plannercore.ErrViewRecursive), IsTrue) + require.True(t, terror.ErrorEqual(err, plannercore.ErrViewRecursive)) tk.MustExec("drop view recursive_view1, t") } -func (s *testSuite6) TestIssue16250(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16250(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists t(a int)") tk.MustExec("create view view_issue16250 as select * from t") _, err := tk.Exec("truncate table view_issue16250") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.view_issue16250' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.view_issue16250' doesn't exist") } -func (s *testSuite6) TestIssue24771(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24771(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists zy_tab;`) tk.MustExec(`create table if not exists zy_tab ( @@ -358,21 +374,25 @@ func (s *testSuite6) TestIssue24771(c *C) { tk.MustQuery(`select * from v_st_2`) } -func (s testSuite6) TestTruncateSequence(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTruncateSequence(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create sequence if not exists seq") _, err := tk.Exec("truncate table seq") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.seq' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.seq' doesn't exist") tk.MustExec("create sequence if not exists seq1 start 10 increment 2 maxvalue 10000 cycle") _, err = tk.Exec("truncate table seq1") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.seq1' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.seq1' doesn't exist") tk.MustExec("drop sequence if exists seq") tk.MustExec("drop sequence if exists seq1") } -func (s *testSuite6) TestCreateViewWithOverlongColName(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateViewWithOverlongColName(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int)") defer tk.MustExec("drop table t") @@ -415,23 +435,25 @@ func (s *testSuite6) TestCreateViewWithOverlongColName(c *C) { tk.MustExec("drop view v ") err := tk.ExecToErr("create view v(`" + strings.Repeat("b", 65) + "`) as select a from t;") - c.Assert(err.Error(), Equals, "[ddl:1059]Identifier name 'bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb' is too long") + require.EqualError(t, err, "[ddl:1059]Identifier name 'bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb' is too long") } -func (s *testSuite6) TestCreateDropDatabase(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateDropDatabase(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists drop_test;") tk.MustExec("drop database if exists drop_test;") tk.MustExec("create database drop_test;") tk.MustExec("use drop_test;") tk.MustExec("drop database drop_test;") _, err := tk.Exec("drop table t;") - c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) + require.Equal(t, plannercore.ErrNoDB.Error(), err.Error()) err = tk.ExecToErr("select * from t;") - c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) + require.Equal(t, plannercore.ErrNoDB.Error(), err.Error()) _, err = tk.Exec("drop database mysql") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("create database charset_test charset ascii;") tk.MustQuery("show create database charset_test;").Check(testutil.RowsWithSep("|", @@ -476,8 +498,10 @@ func (s *testSuite6) TestCreateDropDatabase(c *C) { )) } -func (s *testSuite6) TestCreateDropTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateDropTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists drop_test (a int)") tk.MustExec("drop table if exists drop_test") @@ -485,29 +509,30 @@ func (s *testSuite6) TestCreateDropTable(c *C) { tk.MustExec("drop table drop_test") _, err := tk.Exec("drop table mysql.gc_delete_range") - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testSuite6) TestCreateDropView(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateDropView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create or replace view drop_test as select 1,2") _, err := tk.Exec("drop table drop_test") - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.drop_test'") + require.EqualError(t, err, "[schema:1051]Unknown table 'test.drop_test'") - _, err = tk.Exec("drop view if exists drop_test") - c.Assert(err, IsNil) + tk.MustExec("drop view if exists drop_test") _, err = tk.Exec("drop view mysql.gc_delete_range") - c.Assert(err.Error(), Equals, "Drop tidb system table 'mysql.gc_delete_range' is forbidden") + require.EqualError(t, err, "Drop tidb system table 'mysql.gc_delete_range' is forbidden") _, err = tk.Exec("drop view drop_test") - c.Assert(err.Error(), Equals, "[schema:1051]Unknown table 'test.drop_test'") + require.EqualError(t, err, "[schema:1051]Unknown table 'test.drop_test'") tk.MustExec("create table t_v(a int)") _, err = tk.Exec("drop view t_v") - c.Assert(err.Error(), Equals, "[ddl:1347]'test.t_v' is not VIEW") + require.EqualError(t, err, "[ddl:1347]'test.t_v' is not VIEW") tk.MustExec("create table t_v1(a int, b int);") tk.MustExec("create table t_v2(a int, b int);") @@ -517,8 +542,10 @@ func (s *testSuite6) TestCreateDropView(c *C) { testkit.Rows("def test v SELECT `test`.`t_v2`.`a` AS `a`,`test`.`t_v2`.`b` AS `b` FROM `test`.`t_v2` CASCADED NO @ DEFINER utf8mb4 utf8mb4_bin")) } -func (s *testSuite6) TestCreateDropIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateDropIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists drop_test (a int)") tk.MustExec("create index idx_a on drop_test (a)") @@ -526,8 +553,10 @@ func (s *testSuite6) TestCreateDropIndex(c *C) { tk.MustExec("drop table drop_test") } -func (s *testSuite6) TestAlterTableAddColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists alter_test (c1 int)") tk.MustExec("insert into alter_test values(1)") @@ -535,87 +564,93 @@ func (s *testSuite6) TestAlterTableAddColumn(c *C) { time.Sleep(1 * time.Millisecond) now := time.Now().Add(-1 * time.Millisecond).Format(types.TimeFormat) r, err := tk.Exec("select c2 from alter_test") - c.Assert(err, IsNil) + require.NoError(t, err) req := r.NewChunk(nil) err = r.Next(context.Background(), req) - c.Assert(err, IsNil) + require.NoError(t, err) row := req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(now, GreaterEqual, row.GetTime(0).String()) - c.Assert(r.Close(), IsNil) + require.Equal(t, 1, row.Len()) + require.GreaterOrEqual(t, now, row.GetTime(0).String()) + require.Nil(t, r.Close()) tk.MustExec("alter table alter_test add column c3 varchar(50) default 'CURRENT_TIMESTAMP'") tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") _, err = tk.Exec("alter table alter_view add column c4 varchar(50)") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop view alter_view") tk.MustExec("create sequence alter_seq") _, err = tk.Exec("alter table alter_seq add column c int") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop sequence alter_seq") } -func (s *testSuite6) TestAlterTableAddColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableAddColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table if not exists alter_test (c1 int)") tk.MustExec("insert into alter_test values(1)") tk.MustExec("alter table alter_test add column c2 timestamp default current_timestamp, add column c8 varchar(50) default 'CURRENT_TIMESTAMP'") tk.MustExec("alter table alter_test add column (c7 timestamp default current_timestamp, c3 varchar(50) default 'CURRENT_TIMESTAMP')") r, err := tk.Exec("select c2 from alter_test") - c.Assert(err, IsNil) + require.NoError(t, err) req := r.NewChunk(nil) err = r.Next(context.Background(), req) - c.Assert(err, IsNil) + require.NoError(t, err) row := req.GetRow(0) - c.Assert(row.Len(), Equals, 1) - c.Assert(r.Close(), IsNil) + require.Equal(t, 1, row.Len()) + require.Nil(t, r.Close()) tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") _, err = tk.Exec("alter table alter_view add column (c4 varchar(50), c5 varchar(50))") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop view alter_view") tk.MustExec("create sequence alter_seq") _, err = tk.Exec("alter table alter_seq add column (c1 int, c2 varchar(10))") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop sequence alter_seq") } -func (s *testSuite6) TestAddNotNullColumnNoDefault(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAddNotNullColumnNoDefault(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table nn (c1 int)") tk.MustExec("insert nn values (1), (2)") tk.MustExec("alter table nn add column c2 int not null") - tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("nn")) - c.Assert(err, IsNil) + tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("nn")) + require.NoError(t, err) col2 := tbl.Meta().Columns[1] - c.Assert(col2.DefaultValue, IsNil) - c.Assert(col2.OriginDefaultValue, Equals, "0") + require.Nil(t, col2.DefaultValue) + require.Equal(t, "0", col2.OriginDefaultValue) tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0")) _, err = tk.Exec("insert nn (c1) values (3)") - c.Check(err, NotNil) + require.Error(t, err) tk.MustExec("set sql_mode=''") tk.MustExec("insert nn (c1) values (3)") tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0", "3 0")) } -func (s *testSuite6) TestAlterTableModifyColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAlterTableModifyColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists mc") tk.MustExec("create table mc(c1 int, c2 varchar(10), c3 bit)") _, err := tk.Exec("alter table mc modify column c1 short") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("alter table mc modify column c1 bigint") _, err = tk.Exec("alter table mc modify column c2 blob") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("alter table mc modify column c2 varchar(8)") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("alter table mc modify column c2 varchar(11)") tk.MustExec("alter table mc modify column c2 text(13)") tk.MustExec("alter table mc modify column c2 text") @@ -623,75 +658,58 @@ func (s *testSuite6) TestAlterTableModifyColumn(c *C) { result := tk.MustQuery("show create table mc") createSQL := result.Rows()[0][1] expected := "CREATE TABLE `mc` (\n `c1` bigint(20) DEFAULT NULL,\n `c2` text DEFAULT NULL,\n `c3` bit(1) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" - c.Assert(createSQL, Equals, expected) + require.Equal(t, expected, createSQL) tk.MustExec("create or replace view alter_view as select c1,c2 from mc") _, err = tk.Exec("alter table alter_view modify column c2 text") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop view alter_view") tk.MustExec("create sequence alter_seq") _, err = tk.Exec("alter table alter_seq modify column c int") - c.Assert(err.Error(), Equals, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + require.Equal(t, ddl.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) tk.MustExec("drop sequence alter_seq") // test multiple collate modification in column. tk.MustExec("drop table if exists modify_column_multiple_collate") tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") _, err = tk.Exec("alter table modify_column_multiple_collate modify column a char(1) collate utf8mb4_bin;") - c.Assert(err, IsNil) - t, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) - c.Assert(err, IsNil) - c.Assert(t.Cols()[0].Charset, Equals, "utf8mb4") - c.Assert(t.Cols()[0].Collate, Equals, "utf8mb4_bin") - c.Assert(t.Meta().Charset, Equals, "utf8mb4") - c.Assert(t.Meta().Collate, Equals, "utf8mb4_bin") + require.NoError(t, err) + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + require.NoError(t, err) + require.Equal(t, "utf8mb4", tt.Cols()[0].Charset) + require.Equal(t, "utf8mb4_bin", tt.Cols()[0].Collate) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) tk.MustExec("drop table if exists modify_column_multiple_collate;") tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") _, err = tk.Exec("alter table modify_column_multiple_collate modify column a char(1) charset utf8mb4 collate utf8mb4_bin;") - c.Assert(err, IsNil) - t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) - c.Assert(err, IsNil) - c.Assert(t.Cols()[0].Charset, Equals, "utf8mb4") - c.Assert(t.Cols()[0].Collate, Equals, "utf8mb4_bin") - c.Assert(t.Meta().Charset, Equals, "utf8mb4") - c.Assert(t.Meta().Collate, Equals, "utf8mb4_bin") + require.NoError(t, err) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + require.NoError(t, err) + require.Equal(t, "utf8mb4", tt.Cols()[0].Charset) + require.Equal(t, "utf8mb4_bin", tt.Cols()[0].Collate) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) // test Err case for multiple collate modification in column. tk.MustExec("drop table if exists err_modify_multiple_collate;") tk.MustExec("create table err_modify_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") _, err = tk.Exec("alter table err_modify_multiple_collate modify column a char(1) charset utf8mb4 collate utf8_bin;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_bin", "utf8mb4").Error()) + require.Error(t, err) + require.Equal(t, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_bin", "utf8mb4").Error(), err.Error()) tk.MustExec("drop table if exists err_modify_multiple_collate;") tk.MustExec("create table err_modify_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") _, err = tk.Exec("alter table err_modify_multiple_collate modify column a char(1) collate utf8_bin collate utf8mb4_bin;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_bin", "utf8").Error()) + require.Error(t, err) + require.Equal(t, ddl.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_bin", "utf8").Error(), err.Error()) } -func (s *testSuite6) TestDefaultDBAfterDropCurDB(c *C) { - tk := testkit.NewTestKit(c, s.store) - - testSQL := `create database if not exists test_db CHARACTER SET latin1 COLLATE latin1_swedish_ci;` - tk.MustExec(testSQL) - - testSQL = `use test_db;` - tk.MustExec(testSQL) - tk.MustQuery(`select database();`).Check(testkit.Rows("test_db")) - tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows("latin1")) - tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("latin1_swedish_ci")) - - testSQL = `drop database test_db;` - tk.MustExec(testSQL) - tk.MustQuery(`select database();`).Check(testkit.Rows("")) - tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows(mysql.DefaultCharset)) - tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows(mysql.DefaultCollationName)) -} - -func (s *testSuite6) TestColumnCharsetAndCollate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestColumnCharsetAndCollate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) dbName := "col_charset_collate" tk.MustExec("create database " + dbName) tk.MustExec("use " + dbName) @@ -752,7 +770,7 @@ func (s *testSuite6) TestColumnCharsetAndCollate(c *C) { errMsg: "", }, } - sctx := tk.Se.(sessionctx.Context) + sctx := tk.Session() dm := domain.GetDomain(sctx) for i, tt := range tests { tblName := fmt.Sprintf("t%d", i) @@ -760,29 +778,31 @@ func (s *testSuite6) TestColumnCharsetAndCollate(c *C) { if tt.errMsg == "" { tk.MustExec(sql) is := dm.InfoSchema() - c.Assert(is, NotNil) + require.NotNil(t, is) tb, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) - c.Assert(err, IsNil) - c.Assert(tb.Meta().Columns[0].Charset, Equals, tt.exptCharset, Commentf(sql)) - c.Assert(tb.Meta().Columns[0].Collate, Equals, tt.exptCollate, Commentf(sql)) + require.NoError(t, err) + require.Equalf(t, tt.exptCharset, tb.Meta().Columns[0].Charset, sql) + require.Equalf(t, tt.exptCollate, tb.Meta().Columns[0].Collate, sql) } else { _, err := tk.Exec(sql) - c.Assert(err, NotNil, Commentf(sql)) + require.Errorf(t, err, sql) } } tk.MustExec("drop database " + dbName) } -func (s *testSuite6) TestTooLargeIdentifierLength(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTooLargeIdentifierLength(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // for database. dbName1, dbName2 := strings.Repeat("a", mysql.MaxDatabaseNameLength), strings.Repeat("a", mysql.MaxDatabaseNameLength+1) tk.MustExec(fmt.Sprintf("create database %s", dbName1)) tk.MustExec(fmt.Sprintf("drop database %s", dbName1)) _, err := tk.Exec(fmt.Sprintf("create database %s", dbName2)) - c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", dbName2)) + require.Equal(t, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", dbName2), err.Error()) // for table. tk.MustExec("use test") @@ -790,7 +810,7 @@ func (s *testSuite6) TestTooLargeIdentifierLength(c *C) { tk.MustExec(fmt.Sprintf("create table %s(c int)", tableName1)) tk.MustExec(fmt.Sprintf("drop table %s", tableName1)) _, err = tk.Exec(fmt.Sprintf("create table %s(c int)", tableName2)) - c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", tableName2)) + require.Equal(t, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", tableName2), err.Error()) // for column. tk.MustExec("drop table if exists t;") @@ -798,7 +818,7 @@ func (s *testSuite6) TestTooLargeIdentifierLength(c *C) { tk.MustExec(fmt.Sprintf("create table t(%s int)", columnName1)) tk.MustExec("drop table t") _, err = tk.Exec(fmt.Sprintf("create table t(%s int)", columnName2)) - c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", columnName2)) + require.Equal(t, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", columnName2), err.Error()) // for index. tk.MustExec("create table t(c int);") @@ -806,16 +826,18 @@ func (s *testSuite6) TestTooLargeIdentifierLength(c *C) { tk.MustExec(fmt.Sprintf("create index %s on t(c)", indexName1)) tk.MustExec(fmt.Sprintf("drop index %s on t", indexName1)) _, err = tk.Exec(fmt.Sprintf("create index %s on t(c)", indexName2)) - c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", indexName2)) + require.Equal(t, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", indexName2), err.Error()) // for create table with index. tk.MustExec("drop table t;") _, err = tk.Exec(fmt.Sprintf("create table t(c int, index %s(c));", indexName2)) - c.Assert(err.Error(), Equals, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", indexName2)) + require.Equal(t, fmt.Sprintf("[ddl:1059]Identifier name '%s' is too long", indexName2), err.Error()) } -func (s *testSuite8) TestShardRowIDBits(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShardRowIDBits(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int) shard_row_id_bits = 15") @@ -823,16 +845,16 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { tk.MustExec("insert into t values (?)", i) } - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) - assertCountAndShard := func(t table.Table, expectCount int) { + assertCountAndShard := func(tt table.Table, expectCount int) { var hasShardedID bool var count int - c.Assert(tk.Se.NewTxn(context.Background()), IsNil) - err = tables.IterRecords(t, tk.Se, nil, func(h kv.Handle, rec []types.Datum, cols []*table.Column) (more bool, err error) { - c.Assert(h.IntValue(), GreaterEqual, int64(0)) + require.NoError(t, tk.Session().NewTxn(context.Background())) + err = tables.IterRecords(tt, tk.Session(), nil, func(h kv.Handle, rec []types.Datum, cols []*table.Column) (more bool, err error) { + require.GreaterOrEqual(t, h.IntValue(), int64(0)) first8bits := h.IntValue() >> 56 if first8bits > 0 { hasShardedID = true @@ -840,9 +862,9 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { count++ return true, nil }) - c.Assert(err, IsNil) - c.Assert(count, Equals, expectCount) - c.Assert(hasShardedID, IsTrue) + require.NoError(t, err) + require.Equal(t, expectCount, count) + require.True(t, hasShardedID) } assertCountAndShard(tbl, 100) @@ -876,22 +898,22 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { // Hack an existing table with shard_row_id_bits and primary key as handle db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test")) - c.Assert(ok, IsTrue) + require.True(t, ok) tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) tblInfo := tbl.Meta() tblInfo.ShardRowIDBits = 5 tblInfo.MaxShardRowIDBits = 5 - err = kv.RunInNewTxn(context.Background(), s.store, false, func(ctx context.Context, txn kv.Transaction) error { + err = kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { m := meta.NewMeta(txn) _, err = m.GenSchemaVersion() - c.Assert(err, IsNil) - c.Assert(m.UpdateTable(db.ID, tblInfo), IsNil) + require.NoError(t, err) + require.Nil(t, m.UpdateTable(db.ID, tblInfo)) return nil }) - c.Assert(err, IsNil) + require.NoError(t, err) err = dom.Reload() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("insert auto(b) values (1), (3), (5)") tk.MustQuery("select id from auto order by id").Check(testkit.Rows("1", "2", "3")) @@ -907,11 +929,11 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) assertCountAndShard(tbl, 100) prevB, err := strconv.Atoi(tk.MustQuery("select b from auto where a=0").Rows()[0][0].(string)) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 1; i < 100; i++ { b, err := strconv.Atoi(tk.MustQuery(fmt.Sprintf("select b from auto where a=%d", i)).Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(b, Greater, prevB) + require.NoError(t, err) + require.Greater(t, b, prevB) prevB = b } @@ -921,26 +943,28 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { defer tk.MustExec("drop table if exists t1") tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) maxID := 1<<(64-15-1) - 1 - alloc := tbl.Allocators(tk.Se).Get(autoid.RowIDAllocType) + alloc := tbl.Allocators(tk.Session()).Get(autoid.RowIDAllocType) err = alloc.Rebase(context.Background(), int64(maxID)-1, false) - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("insert into t1 values(1)") // continue inserting will fail. _, err = tk.Exec("insert into t1 values(2)") - c.Assert(autoid.ErrAutoincReadFailed.Equal(err), IsTrue, Commentf("err:%v", err)) + require.Truef(t, autoid.ErrAutoincReadFailed.Equal(err), "err:%v", err) _, err = tk.Exec("insert into t1 values(3)") - c.Assert(autoid.ErrAutoincReadFailed.Equal(err), IsTrue, Commentf("err:%v", err)) + require.Truef(t, autoid.ErrAutoincReadFailed.Equal(err), "err:%v", err) } type testAutoRandomSuite struct { *baseTestSuite } -func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomBitsData(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists test_auto_random_bits") defer tk.MustExec("drop database if exists test_auto_random_bits") @@ -948,8 +972,8 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { tk.MustExec("drop table if exists t") extractAllHandles := func() []int64 { - allHds, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test_auto_random_bits", "t") - c.Assert(err, IsNil) + allHds, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test_auto_random_bits", "t") + require.NoError(t, err) return allHds } @@ -963,18 +987,18 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { tk.MustExec("drop table t") // Test auto random id number. - c.Assert(len(allHandles), Equals, 100) + require.Equal(t, 100, len(allHandles)) // Test the handles are not all zero. allZero := true for _, h := range allHandles { allZero = allZero && (h>>(64-16)) == 0 } - c.Assert(allZero, IsFalse) + require.False(t, allZero) // Test non-shard-bits part of auto random id is monotonic increasing and continuous. orderedHandles := testutil.MaskSortHandles(allHandles, 15, mysql.TypeLonglong) size := int64(len(allHandles)) for i := int64(1); i <= size; i++ { - c.Assert(i, Equals, orderedHandles[i-1]) + require.Equal(t, orderedHandles[i-1], i) } // Test explicit insert. @@ -984,8 +1008,8 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i+autoRandBitsUpperBound, i)) } _, err := tk.Exec("insert into t (b) values (0)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + require.Error(t, err) + require.Equal(t, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error(), err.Error()) tk.MustExec("drop table t") // Test overflow. @@ -994,16 +1018,16 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { // so firstly we rebase auto_rand to the position before overflow. tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", autoRandBitsUpperBound, 1)) _, err = tk.Exec("insert into t (b) values (0)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + require.Error(t, err) + require.Equal(t, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error(), err.Error()) tk.MustExec("drop table t") tk.MustExec("create table t (a bigint primary key auto_random(15), b int)") tk.MustExec("insert into t values (1, 2)") tk.MustExec(fmt.Sprintf("update t set a = %d where a = 1", autoRandBitsUpperBound)) _, err = tk.Exec("insert into t (b) values (0)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + require.Error(t, err) + require.Equal(t, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error(), err.Error()) tk.MustExec("drop table t") // Test insert negative integers explicitly won't trigger rebase. @@ -1016,10 +1040,10 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { orderedHandles = testutil.MaskSortHandles(extractAllHandles(), 15, mysql.TypeLonglong) size = int64(len(allHandles)) for i := int64(0); i < 100; i++ { - c.Assert(orderedHandles[i], Equals, i-100) + require.Equal(t, i-100, orderedHandles[i]) } for i := int64(100); i < size; i++ { - c.Assert(orderedHandles[i], Equals, i-99) + require.Equal(t, i-99, orderedHandles[i]) } tk.MustExec("drop table t") @@ -1030,7 +1054,7 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { } for _, h := range extractAllHandles() { // Sign bit should be reserved. - c.Assert(h > 0, IsTrue) + require.True(t, h > 0) } tk.MustExec("drop table t") @@ -1043,7 +1067,7 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { signBitUnused = signBitUnused && (h > 0) } // Sign bit should be used for shard. - c.Assert(signBitUnused, IsFalse) + require.False(t, signBitUnused) tk.MustExec("drop table t;") // Test rename table does not affect incremental part of auto_random ID. @@ -1064,44 +1088,46 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { for _, h := range extractAllHandles() { uniqueHandles[h&((1<<(63-5))-1)] = struct{}{} } - c.Assert(len(uniqueHandles), Equals, 30) + require.Equal(t, 30, len(uniqueHandles)) tk.MustExec("drop database test_auto_random_bits_rename;") tk.MustExec("drop table t;") } -func (s *testAutoRandomSuite) TestAutoRandomTableOption(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomTableOption(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // test table option is auto-random tk.MustExec("drop table if exists auto_random_table_option") tk.MustExec("create table auto_random_table_option (a bigint auto_random(5) key) auto_random_base = 1000") - t, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) - c.Assert(err, IsNil) - c.Assert(t.Meta().AutoRandID, Equals, int64(1000)) + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) + require.NoError(t, err) + require.Equal(t, int64(1000), tt.Meta().AutoRandID) tk.MustExec("insert into auto_random_table_option values (),(),(),(),()") - allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "auto_random_table_option") - c.Assert(err, IsNil) - c.Assert(len(allHandles), Equals, 5) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "auto_random_table_option") + require.NoError(t, err) + require.Equal(t, 5, len(allHandles)) // Test non-shard-bits part of auto random id is monotonic increasing and continuous. orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) size := int64(len(allHandles)) for i := int64(0); i < size; i++ { - c.Assert(i+1000, Equals, orderedHandles[i]) + require.Equal(t, orderedHandles[i], i+1000) } tk.MustExec("drop table if exists alter_table_auto_random_option") tk.MustExec("create table alter_table_auto_random_option (a bigint primary key auto_random(4), b int)") - t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) - c.Assert(err, IsNil) - c.Assert(t.Meta().AutoRandID, Equals, int64(0)) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + require.NoError(t, err) + require.Equal(t, int64(0), tt.Meta().AutoRandID) tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") - c.Assert(err, IsNil) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "alter_table_auto_random_option") + require.NoError(t, err) orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) size = int64(len(allHandles)) for i := int64(0); i < size; i++ { - c.Assert(orderedHandles[i], Equals, i+1) + require.Equal(t, i+1, orderedHandles[i]) } tk.MustExec("delete from alter_table_auto_random_option") @@ -1110,24 +1136,24 @@ func (s *testAutoRandomSuite) TestAutoRandomTableOption(c *C) { // value is not what we rebased, because the local cache is dropped, here we choose // a quite big value to do this. tk.MustExec("alter table alter_table_auto_random_option auto_random_base = 3000000") - t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) - c.Assert(err, IsNil) - c.Assert(t.Meta().AutoRandID, Equals, int64(3000000)) + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + require.NoError(t, err) + require.Equal(t, int64(3000000), tt.Meta().AutoRandID) tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") - c.Assert(err, IsNil) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "alter_table_auto_random_option") + require.NoError(t, err) orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) size = int64(len(allHandles)) for i := int64(0); i < size; i++ { - c.Assert(orderedHandles[i], Equals, i+3000000) + require.Equal(t, i+3000000, orderedHandles[i]) } tk.MustExec("drop table alter_table_auto_random_option") // Alter auto_random_base on non auto_random table. tk.MustExec("create table alter_auto_random_normal (a int)") _, err = tk.Exec("alter table alter_auto_random_normal auto_random_base = 100") - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), autoid.AutoRandomRebaseNotApplicable), IsTrue, Commentf(err.Error())) + require.Error(t, err) + require.Contains(t, err.Error(), autoid.AutoRandomRebaseNotApplicable) } // Test filter different kind of allocators. @@ -1136,8 +1162,10 @@ func (s *testAutoRandomSuite) TestAutoRandomTableOption(c *C) { // 2: ActionRebaseAutoID : it will drop row-id-type allocator. // 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. // 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. -func (s *testAutoRandomSuite) TestFilterDifferentAllocators(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFilterDifferentAllocators(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists t1") @@ -1145,33 +1173,33 @@ func (s *testAutoRandomSuite) TestFilterDifferentAllocators(c *C) { tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)") tk.MustExec("insert into t values()") tk.MustQuery("select b from t").Check(testkit.Rows("1")) - allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") - c.Assert(err, IsNil) - c.Assert(len(allHandles), Equals, 1) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - c.Assert(orderedHandles[0], Equals, int64(1)) + require.Equal(t, int64(1), orderedHandles[0]) tk.MustExec("delete from t") // Test rebase auto_increment. tk.MustExec("alter table t auto_increment 3000000") tk.MustExec("insert into t values()") tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") - c.Assert(err, IsNil) - c.Assert(len(allHandles), Equals, 1) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - c.Assert(orderedHandles[0], Equals, int64(2)) + require.Equal(t, int64(2), orderedHandles[0]) tk.MustExec("delete from t") // Test rebase auto_random. tk.MustExec("alter table t auto_random_base 3000000") tk.MustExec("insert into t values()") tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") - c.Assert(err, IsNil) - c.Assert(len(allHandles), Equals, 1) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - c.Assert(orderedHandles[0], Equals, int64(3000000)) + require.Equal(t, int64(3000000), orderedHandles[0]) tk.MustExec("delete from t") // Test rename table. @@ -1179,17 +1207,19 @@ func (s *testAutoRandomSuite) TestFilterDifferentAllocators(c *C) { tk.MustExec("insert into t1 values()") res := tk.MustQuery("select b from t1") strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) - c.Assert(err, IsNil) - c.Assert(strInt64, Greater, int64(3000002)) - allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t1") - c.Assert(err, IsNil) - c.Assert(len(allHandles), Equals, 1) + require.NoError(t, err) + require.Greater(t, strInt64, int64(3000002)) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t1") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) - c.Assert(orderedHandles[0], Greater, int64(3000001)) + require.Greater(t, orderedHandles[0], int64(3000001)) } -func (s *testSuite6) TestMaxHandleAddIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMaxHandleAddIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a bigint PRIMARY KEY, b int)") @@ -1205,26 +1235,28 @@ func (s *testSuite6) TestMaxHandleAddIndex(c *C) { tk.MustExec("admin check table t1") } -func (s *testSerialSuite) TestSetDDLReorgWorkerCnt(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetDDLReorgWorkerCnt(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(variable.DefTiDBDDLReorgWorkerCount)) + err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(variable.DefTiDBDDLReorgWorkerCount), variable.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 1") - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(1)) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(1), variable.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(100)) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) _, err = tk.Exec("set @@global.tidb_ddl_reorg_worker_cnt = invalid_val") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) + require.Truef(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgWorkerCounter(), Equals, int32(100)) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = -1") tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '-1'")) tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("1")) @@ -1244,29 +1276,31 @@ func (s *testSerialSuite) TestSetDDLReorgWorkerCnt(c *C) { tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("256")) } -func (s *testSerialSuite) TestSetDDLReorgBatchSize(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetDDLReorgBatchSize(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, int32(variable.DefTiDBDDLReorgBatchSize)) + err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(variable.DefTiDBDDLReorgBatchSize), variable.GetDDLReorgBatchSize()) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '1'")) - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, variable.MinDDLReorgBatchSize) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, variable.MinDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", variable.MaxDDLReorgBatchSize+1)) tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '%d'", variable.MaxDDLReorgBatchSize+1))) - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, variable.MaxDDLReorgBatchSize) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, variable.MaxDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) _, err = tk.Exec("set @@global.tidb_ddl_reorg_batch_size = invalid_val") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) + require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") - err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLReorgBatchSize(), Equals, int32(100)) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgBatchSize()) tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '-1'")) @@ -1281,77 +1315,81 @@ func (s *testSerialSuite) TestSetDDLReorgBatchSize(c *C) { res.Check(testkit.Rows("1000")) } -func (s *testSuite6) TestIllegalFunctionCall4GeneratedColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIllegalFunctionCall4GeneratedColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Test create an exist database _, err := tk.Exec("CREATE database test") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("create table t1 (b double generated always as (rand()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error(), err.Error()) _, err = tk.Exec("create table t1 (a varchar(64), b varchar(1024) generated always as (load_file(a)) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error(), err.Error()) _, err = tk.Exec("create table t1 (a datetime generated always as (curdate()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error(), err.Error()) _, err = tk.Exec("create table t1 (a datetime generated always as (current_time()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error(), err.Error()) _, err = tk.Exec("create table t1 (a datetime generated always as (current_timestamp()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error(), err.Error()) _, err = tk.Exec("create table t1 (a datetime, b varchar(10) generated always as (localtime()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error(), err.Error()) _, err = tk.Exec("create table t1 (a varchar(1024) generated always as (uuid()) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("a").Error(), err.Error()) _, err = tk.Exec("create table t1 (a varchar(1024), b varchar(1024) generated always as (is_free_lock(a)) virtual);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("b").Error(), err.Error()) tk.MustExec("create table t1 (a bigint not null primary key auto_increment, b bigint, c bigint as (b + 1));") _, err = tk.Exec("alter table t1 add column d varchar(1024) generated always as (database());") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error(), err.Error()) tk.MustExec("alter table t1 add column d bigint generated always as (b + 1); ") _, err = tk.Exec("alter table t1 modify column d bigint generated always as (connection_id());") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("d").Error(), err.Error()) _, err = tk.Exec("alter table t1 change column c cc bigint generated always as (connection_id());") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("cc").Error()) + require.Equal(t, ddl.ErrGeneratedColumnFunctionIsNotAllowed.GenWithStackByArgs("cc").Error(), err.Error()) } -func (s *testSuite6) TestGeneratedColumnRelatedDDL(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGeneratedColumnRelatedDDL(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Test create an exist database _, err := tk.Exec("CREATE database test") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("create table t1 (a bigint not null primary key auto_increment, b bigint as (a + 1));") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("b").Error()) + require.Equal(t, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("b").Error(), err.Error()) tk.MustExec("create table t1 (a bigint not null primary key auto_increment, b bigint, c bigint as (b + 1));") _, err = tk.Exec("alter table t1 add column d bigint generated always as (a + 1);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("d").Error()) + require.Equal(t, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("d").Error(), err.Error()) tk.MustExec("alter table t1 add column d bigint generated always as (b + 1);") _, err = tk.Exec("alter table t1 modify column d bigint generated always as (a + 1);") - c.Assert(err.Error(), Equals, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("d").Error()) + require.Equal(t, ddl.ErrGeneratedColumnRefAutoInc.GenWithStackByArgs("d").Error(), err.Error()) // This mysql compatibility check can be disabled using tidb_enable_auto_increment_in_generated tk.MustExec("set session tidb_enable_auto_increment_in_generated = 1;") tk.MustExec("alter table t1 modify column d bigint generated always as (a + 1);") _, err = tk.Exec("alter table t1 add column e bigint as (z + 1);") - c.Assert(err.Error(), Equals, ddl.ErrBadField.GenWithStackByArgs("z", "generated column function").Error()) + require.Equal(t, ddl.ErrBadField.GenWithStackByArgs("z", "generated column function").Error(), err.Error()) tk.MustExec("drop table t1;") @@ -1362,37 +1400,41 @@ func (s *testSuite6) TestGeneratedColumnRelatedDDL(c *C) { tk.MustQuery("select * from t1").Check(testkit.Rows("1 2 3")) } -func (s *testSuite6) TestSetDDLErrorCountLimit(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetDDLErrorCountLimit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - err := ddlutil.LoadDDLVars(tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLErrorCountLimit(), Equals, int64(variable.DefTiDBDDLErrorCountLimit)) + err := ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(variable.DefTiDBDDLErrorCountLimit), variable.GetDDLErrorCountLimit()) tk.MustExec("set @@global.tidb_ddl_error_count_limit = -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '-1'")) - err = ddlutil.LoadDDLVars(tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLErrorCountLimit(), Equals, int64(0)) + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(0), variable.GetDDLErrorCountLimit()) tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", uint64(math.MaxInt64)+1)) tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '%d'", uint64(math.MaxInt64)+1))) - err = ddlutil.LoadDDLVars(tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLErrorCountLimit(), Equals, int64(math.MaxInt64)) + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(math.MaxInt64), variable.GetDDLErrorCountLimit()) _, err = tk.Exec("set @@global.tidb_ddl_error_count_limit = invalid_val") - c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue, Commentf("err %v", err)) + require.True(t, terror.ErrorEqual(err, variable.ErrWrongTypeForVar), "err %v", err) tk.MustExec("set @@global.tidb_ddl_error_count_limit = 100") - err = ddlutil.LoadDDLVars(tk.Se) - c.Assert(err, IsNil) - c.Assert(variable.GetDDLErrorCountLimit(), Equals, int64(100)) + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(100), variable.GetDDLErrorCountLimit()) res := tk.MustQuery("select @@global.tidb_ddl_error_count_limit") res.Check(testkit.Rows("100")) } // Test issue #9205, fix the precision problem for time type default values // See https://github.com/pingcap/tidb/issues/9205 for details -func (s *testSuite6) TestIssue9205(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue9205(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(c time DEFAULT '12:12:12.8');`) @@ -1430,48 +1472,52 @@ func (s *testSuite6) TestIssue9205(c *C) { )) } -func (s *testSuite6) TestCheckDefaultFsp(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCheckDefaultFsp(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) _, err := tk.Exec("create table t ( tt timestamp default now(1));") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") _, err = tk.Exec("create table t ( tt timestamp(1) default current_timestamp);") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") _, err = tk.Exec("create table t ( tt timestamp(1) default now(2));") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") tk.MustExec("create table t ( tt timestamp(1) default now(1));") tk.MustExec("create table t2 ( tt timestamp default current_timestamp());") tk.MustExec("create table t3 ( tt timestamp default current_timestamp(0));") _, err = tk.Exec("alter table t add column ttt timestamp default now(2);") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'ttt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") _, err = tk.Exec("alter table t add column ttt timestamp(5) default current_timestamp;") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'ttt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") _, err = tk.Exec("alter table t add column ttt timestamp(5) default now(2);") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'ttt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'ttt'") _, err = tk.Exec("alter table t modify column tt timestamp(1) default now();") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") _, err = tk.Exec("alter table t modify column tt timestamp(4) default now(5);") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tt'") _, err = tk.Exec("alter table t change column tt tttt timestamp(4) default now(5);") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tttt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tttt'") _, err = tk.Exec("alter table t change column tt tttt timestamp(1) default now();") - c.Assert(err.Error(), Equals, "[ddl:1067]Invalid default value for 'tttt'") + require.EqualError(t, err, "[ddl:1067]Invalid default value for 'tttt'") } -func (s *testSuite6) TestTimestampMinDefaultValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTimestampMinDefaultValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tdv;") tk.MustExec("create table tdv(a int);") @@ -1479,12 +1525,14 @@ func (s *testSuite6) TestTimestampMinDefaultValue(c *C) { } // this test will change the fail-point `mockAutoIDChange`, so we move it to the `testRecoverTable` suite -func (s *testRecoverTable) TestRenameTable(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) +func TestRenameTable(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")) }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists rename1") tk.MustExec("drop database if exists rename2") @@ -1538,34 +1586,38 @@ func (s *testRecoverTable) TestRenameTable(c *C) { result = tk.MustQuery("select * from rename2.t1") result.Check(testkit.Rows("1", "100000", "100001")) _, err := tk.Exec("insert rename1.t values ()") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("drop database rename1") tk.MustExec("drop database rename2") } -func (s *testSuite6) TestAutoIncrementColumnErrorMessage(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoIncrementColumnErrorMessage(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Test create an exist database _, err := tk.Exec("CREATE database test") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("CREATE TABLE t1 (t1_id INT NOT NULL AUTO_INCREMENT PRIMARY KEY);") _, err = tk.Exec("CREATE INDEX idx1 ON t1 ((t1_id + t1_id));") - c.Assert(err.Error(), Equals, ddl.ErrExpressionIndexCanNotRefer.GenWithStackByArgs("idx1").Error()) + require.Equal(t, ddl.ErrExpressionIndexCanNotRefer.GenWithStackByArgs("idx1").Error(), err.Error()) // This mysql compatibility check can be disabled using tidb_enable_auto_increment_in_generated tk.MustExec("SET SESSION tidb_enable_auto_increment_in_generated = 1;") tk.MustExec("CREATE INDEX idx1 ON t1 ((t1_id + t1_id));") } -func (s *testRecoverTable) TestRenameMultiTables(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) +func TestRenameMultiTables(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")) }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists rename1") tk.MustExec("drop database if exists rename2") @@ -1612,7 +1664,7 @@ func (s *testRecoverTable) TestRenameMultiTables(c *C) { tk.MustExec("use rename3") tk.MustExec("create table rename3.t3 (a int primary key auto_increment)") tk.MustGetErrCode("rename table rename1.t1 to rename1.t2, rename1.t1 to rename3.t3", errno.ErrTableExists) - tk.MustGetErrCode("rename table rename1.t1 to rename1.t2, rename1.t1 to rename3.t4", errno.ErrFileNotFound) + tk.MustGetErrCode("rename table rename1.t1 to rename1.t2, rename1.t1 to rename3.t4", errno.ErrNoSuchTable) tk.MustExec("drop database rename1") tk.MustExec("drop database rename2") tk.MustExec("drop database rename3") diff --git a/executor/distsql.go b/executor/distsql.go index 92cf0b8d60344..424f47f0ee7b5 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -21,6 +21,7 @@ import ( "runtime" "runtime/trace" "sort" + "strings" "sync" "sync/atomic" "time" @@ -48,6 +49,7 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" @@ -194,11 +196,20 @@ type IndexReaderExecutor struct { memTracker *memory.Tracker selectResultHook // for testing + + // If dummy flag is set, this is not a real IndexReader, it just provides the KV ranges for UnionScan. + // Used by the temporary table, cached table. + dummy bool +} + +// Table implements the dataSourceExecutor interface. +func (e *IndexReaderExecutor) Table() table.Table { + return e.table } // Close clears all resources hold by current object. func (e *IndexReaderExecutor) Close() (err error) { - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { return nil } @@ -212,7 +223,7 @@ func (e *IndexReaderExecutor) Close() (err error) { // Next implements the Executor Next interface. func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error { - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { req.Reset() return nil } @@ -282,16 +293,13 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) // Treat temporary table as dummy table, avoid sending distsql request to TiKV. // In a test case IndexReaderExecutor is mocked and e.table is nil. // Avoid sending distsql request to TIKV. - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { return nil } e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) var builder distsql.RequestBuilder - if e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - builder.SetIsolationLevel(kv.RC) - } builder.SetKeyRanges(kvRanges). SetDAGRequest(e.dagPB). SetStartTS(e.startTS). @@ -383,6 +391,10 @@ type IndexLookUpExecutor struct { // cancelFunc is called when close the executor cancelFunc context.CancelFunc + + // If dummy flag is set, this is not a real IndexLookUpReader, it just provides the KV ranges for UnionScan. + // Used by the temporary table, cached table. + dummy bool } type getHandleType int8 @@ -398,6 +410,11 @@ type checkIndexValue struct { idxTblCols []*table.Column } +// Table implements the dataSourceExecutor interface. +func (e *IndexLookUpExecutor) Table() table.Table { + return e.table +} + // Open implements the Executor Open interface. func (e *IndexLookUpExecutor) Open(ctx context.Context) error { var err error @@ -414,7 +431,7 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { } // Treat temporary table as dummy table, avoid sending distsql request to TiKV. - if e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { return nil } @@ -559,9 +576,6 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< PushedLimit: e.PushedLimit, } var builder distsql.RequestBuilder - if e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - builder.SetIsolationLevel(kv.RC) - } builder.SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). @@ -683,7 +697,7 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup // Close implements Exec Close interface. func (e *IndexLookUpExecutor) Close() error { - if e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { return nil } @@ -711,7 +725,7 @@ func (e *IndexLookUpExecutor) Close() error { // Next implements Exec Next interface. func (e *IndexLookUpExecutor) Next(ctx context.Context, req *chunk.Chunk) error { - if e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { req.Reset() return nil } @@ -1145,15 +1159,52 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta collators = append(collators, collate.GetCollator(tp.Collate)) } + ir := func() *consistency.Reporter { + return &consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(w.idxLookup.table.RecordPrefix(), handle) + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + var idx table.Index + for _, v := range w.idxLookup.table.Indices() { + if strings.EqualFold(v.Meta().Name.String(), w.idxLookup.index.Name.O) { + idx = v + break + } + } + if idx == nil { + return nil + } + k, _, err := idx.GenIndexKey(w.idxLookup.ctx.GetSessionVars().StmtCtx, idxRow.Values[:len(idx.Meta().Columns)], idxRow.Handle, nil) + if err != nil { + return nil + } + return k + }, + Tbl: tblInfo, + Idx: w.idxLookup.index, + Sctx: w.idxLookup.ctx, + } + } + for { err := Next(ctx, tableReader, chk) if err != nil { return errors.Trace(err) } + + // If ctx is cancelled, `Next` may return empty result when the actual data is not empty. To avoid producing + // false-positive error logs that cause confusion, exit in this case. + select { + case <-ctx.Done(): + return nil + default: + } + if chk.NumRows() == 0 { task.indexOrder.Range(func(h kv.Handle, val interface{}) bool { idxRow := task.idxRows.GetRow(val.(int)) - err = ErrDataInConsistentExtraIndex.GenWithStackByArgs(h, idxRow.GetDatum(0, w.idxColTps[0]), nil) + err = ir().ReportAdminCheckInconsistent(ctx, h, &consistency.RecordData{Handle: h, Values: getDatumRow(&idxRow, w.idxColTps)}, nil) return false }) if err != nil { @@ -1188,12 +1239,32 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo) cmpRes, err := idxVal.Compare(sctx, &vals[i], collators[i]) if err != nil { - return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), vals[i], err) + fts := make([]*types.FieldType, 0, len(w.idxTblCols)) + for _, c := range w.idxTblCols { + fts = append(fts, &c.FieldType) + } + return ir().ReportAdminCheckInconsistentWithColInfo(ctx, + handle, + col.Name.O, + idxRow.GetDatum(i, tp), + vals[i], + err, + &consistency.RecordData{Handle: handle, Values: getDatumRow(&idxRow, fts)}, + ) } if cmpRes != 0 { - return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, - handle, idxRow.GetDatum(i, tp), vals[i], err) + fts := make([]*types.FieldType, 0, len(w.idxTblCols)) + for _, c := range w.idxTblCols { + fts = append(fts, &c.FieldType) + } + return ir().ReportAdminCheckInconsistentWithColInfo(ctx, + handle, + col.Name.O, + idxRow.GetDatum(i, tp), + vals[i], + err, + &consistency.RecordData{Handle: handle, Values: getDatumRow(&idxRow, fts)}, + ) } } } @@ -1201,6 +1272,18 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta return nil } +func getDatumRow(r *chunk.Row, fields []*types.FieldType) []types.Datum { + datumRow := make([]types.Datum, 0, r.Chunk().NumCols()) + for colIdx := 0; colIdx < r.Chunk().NumCols(); colIdx++ { + if colIdx >= len(fields) { + break + } + datum := r.GetDatum(colIdx, fields[colIdx]) + datumRow = append(datumRow, datum) + } + return datumRow +} + // executeTask executes the table look up tasks. We will construct a table reader and send request by handles. // Then we hold the returning rows and finish this task. func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) error { @@ -1271,26 +1354,22 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er } obtainedHandlesMap.Set(handle, true) } - - if w.idxLookup.ctx.GetSessionVars().EnableRedactLog { - logutil.Logger(ctx).Error("inconsistent index handles", - zap.String("table_name", w.idxLookup.index.Table.O), - zap.String("index", w.idxLookup.index.Name.O), - zap.Int("index_cnt", handleCnt), - zap.Int("table_cnt", len(task.rows))) - } else { - logutil.Logger(ctx).Error("inconsistent index handles", - zap.String("table_name", w.idxLookup.index.Table.O), - zap.String("index", w.idxLookup.index.Name.O), - zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), - zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), - zap.String("total_handles", fmt.Sprint(task.handles))) - } - - // table scan in double read can never has conditions according to convertToIndexScan. - // if this table scan has no condition, the number of rows it returns must equal to the length of handles. - return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", - w.idxLookup.index.Name.O, handleCnt, len(task.rows)) + missHds := GetLackHandles(task.handles, obtainedHandlesMap) + return (&consistency.Reporter{ + HandleEncode: func(hd kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(w.idxLookup.table.RecordPrefix(), hd) + }, + Tbl: w.idxLookup.table.Meta(), + Idx: w.idxLookup.index, + Sctx: w.idxLookup.ctx, + }).ReportLookupInconsistent(ctx, + handleCnt, + len(task.rows), + missHds, + task.handles, + nil, + //missRecords, + ) } } diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 5da10b66e9914..93c31490ea56a 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -24,7 +24,6 @@ import ( "testing" "time" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" @@ -56,7 +55,7 @@ func checkGoroutineExists(keyword string) bool { func TestCopClientSend(t *testing.T) { t.Skip("not stable") var cluster testutils.Cluster - store, clean := testkit.CreateMockStore(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { mockstore.BootstrapWithSingleStore(c) cluster = c })) @@ -78,7 +77,6 @@ func TestCopClientSend(t *testing.T) { tk.MustExec("insert copclient values " + strings.Join(values, ",")) // Get table ID for split. - dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("copclient")) require.NoError(t, err) @@ -258,7 +256,7 @@ func TestInconsistentIndex(t *testing.T) { require.NoError(t, err) err = tk.QueryToErr("select * from t use index(idx_a) where a >= 0") - require.Equal(t, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11), err.Error()) + require.Equal(t, fmt.Sprintf("[executor:8133]data inconsistency in table: t, index: idx_a, index-count:%d != record-count:10", i+11), err.Error()) // if has other conditions, the inconsistent index check doesn't work. err = tk.QueryToErr("select * from t where a>=0 and b<10") require.NoError(t, err) diff --git a/executor/errors.go b/executor/errors.go index f6e0f87d08e6b..1e9c2b8969b00 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -54,17 +54,15 @@ var ( ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges) ErrViewInvalid = dbterror.ClassExecutor.NewStd(mysql.ErrViewInvalid) - ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) - ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) - ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) - ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) - ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) - ErrDataInConsistentExtraIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentExtraIndex) - ErrDataInConsistentMisMatchIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentMisMatchIndex) - ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem) - ErrPluginIsNotLoaded = dbterror.ClassExecutor.NewStd(mysql.ErrPluginIsNotLoaded) - ErrSetPasswordAuthPlugin = dbterror.ClassExecutor.NewStd(mysql.ErrSetPasswordAuthPlugin) - ErrFuncNotEnabled = dbterror.ClassExecutor.NewStdErr(mysql.ErrNotSupportedYet, parser_mysql.Message("%-.32s is not supported. To enable this experimental feature, set '%-.32s' in the configuration file.", nil)) + ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) + ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) + ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) + ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) + ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) + ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem) + ErrPluginIsNotLoaded = dbterror.ClassExecutor.NewStd(mysql.ErrPluginIsNotLoaded) + ErrSetPasswordAuthPlugin = dbterror.ClassExecutor.NewStd(mysql.ErrSetPasswordAuthPlugin) + ErrFuncNotEnabled = dbterror.ClassExecutor.NewStdErr(mysql.ErrNotSupportedYet, parser_mysql.Message("%-.32s is not supported. To enable this experimental feature, set '%-.32s' in the configuration file.", nil)) errUnsupportedFlashbackTmpTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("Recover/flashback table is not supported on temporary tables", nil)) errTruncateWrongInsertValue = dbterror.ClassTable.NewStdErr(mysql.ErrTruncatedWrongValue, parser_mysql.Message("Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %d", nil)) diff --git a/executor/executor.go b/executor/executor.go index c7bd9e4bd7e5a..fa4be26da43ff 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -68,6 +68,7 @@ import ( tikverr "github.com/tikv/client-go/v2/error" tikvstore "github.com/tikv/client-go/v2/kv" tikvutil "github.com/tikv/client-go/v2/util" + atomicutil "go.uber.org/atomic" "go.uber.org/zap" ) @@ -102,6 +103,21 @@ var ( GlobalDiskUsageTracker *disk.Tracker ) +var ( + _ dataSourceExecutor = &TableReaderExecutor{} + _ dataSourceExecutor = &IndexReaderExecutor{} + _ dataSourceExecutor = &IndexLookUpExecutor{} + _ dataSourceExecutor = &IndexMergeReaderExecutor{} +) + +// dataSourceExecutor is a table DataSource converted Executor. +// Currently, there are TableReader/IndexReader/IndexLookUp/IndexMergeReader. +// Note, partition reader is special and the caller should handle it carefully. +type dataSourceExecutor interface { + Executor + Table() table.Table +} + type baseExecutor struct { ctx sessionctx.Context id int @@ -446,6 +462,7 @@ type DDLJobRetriever struct { is infoschema.InfoSchema activeRoles []*auth.RoleIdentity cacheJobs []*model.Job + TZLoc *time.Location } func (e *DDLJobRetriever) initial(txn kv.Transaction) error { @@ -494,8 +511,9 @@ func (e *DDLJobRetriever) appendJobToChunk(req *chunk.Chunk, job *model.Job, che tableName = getTableName(e.is, job.TableID) } - startTime := ts2Time(job.StartTS) - finishTime := ts2Time(finishTS) + createTime := ts2Time(job.StartTS, e.TZLoc) + startTime := ts2Time(job.RealStartTS, e.TZLoc) + finishTime := ts2Time(finishTS, e.TZLoc) // Check the privilege. if checker != nil && !checker.RequestVerification(e.activeRoles, strings.ToLower(schemaName), strings.ToLower(tableName), "", mysql.AllPrivMask) { @@ -510,20 +528,25 @@ func (e *DDLJobRetriever) appendJobToChunk(req *chunk.Chunk, job *model.Job, che req.AppendInt64(5, job.SchemaID) req.AppendInt64(6, job.TableID) req.AppendInt64(7, job.RowCount) - req.AppendTime(8, startTime) - if finishTS > 0 { - req.AppendTime(9, finishTime) + req.AppendTime(8, createTime) + if job.RealStartTS > 0 { + req.AppendTime(9, startTime) } else { req.AppendNull(9) } - req.AppendString(10, job.State.String()) + if finishTS > 0 { + req.AppendTime(10, finishTime) + } else { + req.AppendNull(10) + } + req.AppendString(11, job.State.String()) } -func ts2Time(timestamp uint64) types.Time { - duration := time.Duration(math.Pow10(9-int(types.DefaultFsp))) * time.Nanosecond +func ts2Time(timestamp uint64, loc *time.Location) types.Time { + duration := time.Duration(math.Pow10(9-types.DefaultFsp)) * time.Nanosecond t := model.TSConvert2Time(timestamp) t.Truncate(duration) - return types.NewTime(types.FromGoTime(t), mysql.TypeDatetime, types.DefaultFsp) + return types.NewTime(types.FromGoTime(t.In(loc)), mysql.TypeDatetime, types.DefaultFsp) } // ShowDDLJobQueriesExec represents a show DDL job queries executor. @@ -694,6 +717,7 @@ func (e *CheckTableExec) Open(ctx context.Context) error { // Close implements the Executor Close interface. func (e *CheckTableExec) Close() error { var firstErr error + close(e.exitCh) for _, src := range e.srcs { if err := src.Close(); err != nil && firstErr == nil { firstErr = err @@ -727,19 +751,13 @@ func (e *CheckTableExec) checkIndexHandle(ctx context.Context, src *IndexLookUpE for { err = Next(ctx, src, chk) if err != nil { + e.retCh <- errors.Trace(err) break } if chk.NumRows() == 0 { break } - - select { - case <-e.exitCh: - return nil - default: - } } - e.retCh <- errors.Trace(err) return errors.Trace(err) } @@ -769,45 +787,57 @@ func (e *CheckTableExec) Next(ctx context.Context, req *chunk.Chunk) error { if greater == admin.IdxCntGreater { err = e.checkTableIndexHandle(ctx, e.indexInfos[idxOffset]) } else if greater == admin.TblCntGreater { - err = e.checkTableRecord(idxOffset) - } - if err != nil && admin.ErrDataInConsistent.Equal(err) { - return ErrAdminCheckTable.GenWithStack("%v err:%v", e.table.Meta().Name, err) + err = e.checkTableRecord(ctx, idxOffset) } return errors.Trace(err) } // The number of table rows is equal to the number of index rows. // TODO: Make the value of concurrency adjustable. And we can consider the number of records. - concurrency := 3 - wg := sync.WaitGroup{} - for i := range e.srcs { - wg.Add(1) - go func(num int) { - defer wg.Done() + if len(e.srcs) == 1 { + return e.checkIndexHandle(ctx, e.srcs[0]) + } + taskCh := make(chan *IndexLookUpExecutor, len(e.srcs)) + failure := atomicutil.NewBool(false) + concurrency := mathutil.Min(3, len(e.srcs)) + var wg util.WaitGroupWrapper + for _, src := range e.srcs { + taskCh <- src + } + for i := 0; i < concurrency; i++ { + wg.Run(func() { util.WithRecovery(func() { - err1 := e.checkIndexHandle(ctx, e.srcs[num]) - if err1 != nil { - logutil.Logger(ctx).Info("check index handle failed", zap.Error(err1)) + for { + if fail := failure.Load(); fail { + return + } + select { + case src := <-taskCh: + err1 := e.checkIndexHandle(ctx, src) + if err1 != nil { + failure.Store(true) + logutil.Logger(ctx).Info("check index handle failed", zap.Error(err1)) + return + } + case <-e.exitCh: + return + default: + return + } } }, e.handlePanic) - }(i) - - if (i+1)%concurrency == 0 { - wg.Wait() - } + }) } - - for i := 0; i < len(e.srcs); i++ { - err = <-e.retCh - if err != nil { - return errors.Trace(err) - } + wg.Wait() + select { + case err := <-e.retCh: + return errors.Trace(err) + default: + return nil } - return nil } -func (e *CheckTableExec) checkTableRecord(idxOffset int) error { +func (e *CheckTableExec) checkTableRecord(ctx context.Context, idxOffset int) error { idxInfo := e.indexInfos[idxOffset] txn, err := e.ctx.Txn(true) if err != nil { @@ -815,7 +845,7 @@ func (e *CheckTableExec) checkTableRecord(idxOffset int) error { } if e.table.Meta().GetPartitionInfo() == nil { idx := tables.NewIndex(e.table.Meta().ID, e.table.Meta(), idxInfo) - return admin.CheckRecordAndIndex(e.ctx, txn, e.table, idx) + return admin.CheckRecordAndIndex(ctx, e.ctx, txn, e.table, idx) } info := e.table.Meta().GetPartitionInfo() @@ -823,7 +853,7 @@ func (e *CheckTableExec) checkTableRecord(idxOffset int) error { pid := def.ID partition := e.table.(table.PartitionedTable).GetPartition(pid) idx := tables.NewIndex(def.ID, e.table.Meta(), idxInfo) - if err := admin.CheckRecordAndIndex(e.ctx, txn, partition, idx); err != nil { + if err := admin.CheckRecordAndIndex(ctx, e.ctx, txn, partition, idx); err != nil { return errors.Trace(err) } } @@ -974,10 +1004,10 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } } - return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime), e.keys...) + return doLockKeys(ctx, e.ctx, newLockCtx(e.ctx.GetSessionVars(), lockWaitTime, len(e.keys)), e.keys...) } -func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { +func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64, numKeys int) *tikvstore.LockCtx { lockCtx := tikvstore.NewLockCtx(seVars.TxnCtx.GetForUpdateTS(), lockWaitTime, seVars.StmtCtx.GetLockWaitStartTime()) lockCtx.Killed = &seVars.Killed lockCtx.PessimisticLockWaited = &seVars.StmtCtx.PessimisticLockWaited @@ -1010,6 +1040,9 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc rec := deadlockhistory.ErrDeadlockToDeadlockRecord(deadlock) deadlockhistory.GlobalDeadlockHistory.Push(rec) } + if lockCtx.ForUpdateTS > 0 && seVars.AssertionLevel != variable.AssertionLevelOff { + lockCtx.InitCheckExistence(numKeys) + } return lockCtx } @@ -1708,6 +1741,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.OptimizeTracer = nil sc.OptimizerCETrace = nil + sc.SysdateIsNow = ctx.GetSessionVars().SysdateIsNow + sc.InitMemTracker(memory.LabelForSQLText, vars.MemQuotaQuery) sc.InitDiskTracker(memory.LabelForSQLText, -1) sc.MemTracker.AttachToGlobalTracker(GlobalMemoryUsageTracker) @@ -1753,7 +1788,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.OriginalSQL = s.Text() if explainStmt, ok := s.(*ast.ExplainStmt); ok { sc.InExplainStmt = true - sc.IgnoreExplainIDSuffix = (strings.ToLower(explainStmt.Format) == types.ExplainFormatBrief) + sc.IgnoreExplainIDSuffix = strings.ToLower(explainStmt.Format) == types.ExplainFormatBrief sc.InVerboseExplain = strings.ToLower(explainStmt.Format) == types.ExplainFormatVerbose s = explainStmt.Stmt } diff --git a/executor/executor_test.go b/executor/executor_test.go index 48f57a41dce02..df6f43f786cd9 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -73,7 +73,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/israce" @@ -112,14 +111,10 @@ var _ = Suite(&testSuite2{&baseTestSuite{}}) var _ = Suite(&testSuite3{&baseTestSuite{}}) var _ = Suite(&testSuite4{&baseTestSuite{}}) var _ = Suite(&testSuite5{&baseTestSuite{}}) -var _ = Suite(&testSuiteJoin1{&baseTestSuite{}}) -var _ = Suite(&testSuiteJoin2{&baseTestSuite{}}) var _ = Suite(&testSuiteJoin3{&baseTestSuite{}}) -var _ = SerialSuites(&testSuiteJoinSerial{&baseTestSuite{}}) var _ = Suite(&testSuite6{&baseTestSuite{}}) var _ = Suite(&testSuite7{&baseTestSuite{}}) var _ = Suite(&testSuite8{&baseTestSuite{}}) -var _ = Suite(&testUpdateSuite{}) var _ = Suite(&testPointGetSuite{}) var _ = SerialSuites(&testRecoverTable{}) var _ = SerialSuites(&testMemTableReaderSuite{&testClusterTableBase{}}) @@ -325,14 +320,6 @@ func (s *testSuiteP1) TestShow(c *C) { c.Assert(len(tk.MustQuery("show index in t").Rows()), Equals, 1) c.Assert(len(tk.MustQuery("show index from t").Rows()), Equals, 1) - tk.MustQuery("show charset").Check(testkit.Rows( - "ascii US ASCII ascii_bin 1", - "binary binary binary 1", - "gbk Chinese Internal Code Specification gbk_bin 2", - "latin1 Latin1 latin1_bin 1", - "utf8 UTF-8 Unicode utf8_bin 3", - "utf8mb4 UTF-8 Unicode utf8mb4_bin 4", - )) c.Assert(len(tk.MustQuery("show master status").Rows()), Equals, 1) tk.MustQuery("show create database test_show").Check(testkit.Rows("test_show CREATE DATABASE `test_show` /*!40100 DEFAULT CHARACTER SET utf8mb4 */")) tk.MustQuery("show privileges").Check(testkit.Rows("Alter Tables To alter the table", @@ -439,7 +426,7 @@ func (s *testSuite3) TestAdmin(c *C) { err = r.Next(ctx, req) c.Assert(err, IsNil) row = req.GetRow(0) - c.Assert(row.Len(), Equals, 11) + c.Assert(row.Len(), Equals, 12) txn, err = s.store.Begin() c.Assert(err, IsNil) historyJobs, err := admin.GetHistoryDDLJobs(txn, admin.DefNumHistoryJobs) @@ -455,7 +442,7 @@ func (s *testSuite3) TestAdmin(c *C) { err = r.Next(ctx, req) c.Assert(err, IsNil) row = req.GetRow(0) - c.Assert(row.Len(), Equals, 11) + c.Assert(row.Len(), Equals, 12) c.Assert(row.GetInt64(0), Equals, historyJobs[0].ID) c.Assert(err, IsNil) @@ -582,13 +569,49 @@ func (s *testSuiteP2) TestAdminShowDDLJobs(c *C) { re = tk.MustQuery("admin show ddl jobs 1 where job_type='create table'") row = re.Rows()[0] c.Assert(row[1], Equals, "test_admin_show_ddl_jobs") - c.Assert(row[9], Equals, "") + c.Assert(row[10], Equals, "") // Test the START_TIME and END_TIME field. - re = tk.MustQuery("admin show ddl jobs where job_type = 'create table' and start_time > str_to_date('20190101','%Y%m%d%H%i%s')") + tk.MustExec(`set @@time_zone = 'Asia/Shanghai'`) + re = tk.MustQuery("admin show ddl jobs where end_time is not NULL") row = re.Rows()[0] - c.Assert(row[2], Equals, "t") - c.Assert(row[9], Equals, "") + createTime, err := types.ParseDatetime(nil, row[8].(string)) + c.Assert(err, IsNil) + startTime, err := types.ParseDatetime(nil, row[9].(string)) + c.Assert(err, IsNil) + endTime, err := types.ParseDatetime(nil, row[10].(string)) + c.Assert(err, IsNil) + tk.MustExec(`set @@time_zone = 'Europe/Amsterdam'`) + re = tk.MustQuery("admin show ddl jobs where end_time is not NULL") + row2 := re.Rows()[0] + c.Assert(row[8], Not(Equals), row2[8]) + c.Assert(row[9], Not(Equals), row2[9]) + c.Assert(row[10], Not(Equals), row2[10]) + createTime2, err := types.ParseDatetime(nil, row2[8].(string)) + c.Assert(err, IsNil) + startTime2, err := types.ParseDatetime(nil, row2[9].(string)) + c.Assert(err, IsNil) + endTime2, err := types.ParseDatetime(nil, row2[10].(string)) + c.Assert(err, IsNil) + loc, err := time.LoadLocation("Asia/Shanghai") + c.Assert(err, IsNil) + loc2, err := time.LoadLocation("Europe/Amsterdam") + c.Assert(err, IsNil) + t, err := createTime.GoTime(loc) + c.Assert(err, IsNil) + t2, err := createTime2.GoTime(loc2) + c.Assert(err, IsNil) + c.Assert(t.In(time.UTC), Equals, t2.In(time.UTC)) + t, err = startTime.GoTime(loc) + c.Assert(err, IsNil) + t2, err = startTime2.GoTime(loc2) + c.Assert(err, IsNil) + c.Assert(t.In(time.UTC), Equals, t2.In(time.UTC)) + t, err = endTime.GoTime(loc) + c.Assert(err, IsNil) + t2, err = endTime2.GoTime(loc2) + c.Assert(err, IsNil) + c.Assert(t.In(time.UTC), Equals, t2.In(time.UTC)) } func (s *testSuiteP2) TestAdminShowDDLJobsInfo(c *C) { @@ -3027,6 +3050,22 @@ func (s *testSuite) TestTimestampDefaultValueTimeZone(c *C) { tk.MustExec("admin check table t") tk.MustExec(`set time_zone = '+05:00'`) tk.MustExec("admin check table t") + + // 1. add a timestamp general column + // 2. add the index + tk.MustExec(`drop table if exists t`) + // change timezone + tk.MustExec(`set time_zone = 'Asia/Shanghai'`) + tk.MustExec(`create table t(a timestamp default current_timestamp)`) + tk.MustExec(`insert into t set a=now()`) + tk.MustExec(`alter table t add column b timestamp as (a+1) virtual;`) + // change timezone + tk.MustExec(`set time_zone = '+05:00'`) + tk.MustExec(`insert into t set a=now()`) + tk.MustExec(`alter table t add index(b);`) + tk.MustExec("admin check table t") + tk.MustExec(`set time_zone = '-03:00'`) + tk.MustExec("admin check table t") } func (s *testSuite) TestTiDBCurrentTS(c *C) { @@ -3406,10 +3445,6 @@ type checkRequestClient struct { } } -func (c *checkRequestClient) setCheckPriority(priority kvrpcpb.CommandPri) { - atomic.StoreInt32((*int32)(&c.priority), int32(priority)) -} - func (c *checkRequestClient) getCheckPriority() kvrpcpb.CommandPri { return (kvrpcpb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) } @@ -3495,76 +3530,6 @@ func (s *testSuiteWithCliBase) TearDownTest(c *C) { } } -func (s *testSuite2) TestAddIndexPriority(c *C) { - cli := &checkRequestClient{} - hijackClient := func(c tikv.Client) tikv.Client { - cli.Client = c - return cli - } - - store, err := mockstore.NewMockStore( - mockstore.WithClientHijacker(hijackClient), - ) - c.Assert(err, IsNil) - dom, err := session.BootstrapSession(store) - c.Assert(err, IsNil) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() - - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") - tk.MustExec("create table t1 (id int, v int)") - - // Insert some data to make sure plan build IndexLookup for t1. - for i := 0; i < 10; i++ { - tk.MustExec(fmt.Sprintf("insert into t1 values (%d, %d)", i, i)) - } - - cli.mu.Lock() - cli.mu.checkFlags = checkDDLAddIndexPriority - cli.mu.Unlock() - - cli.setCheckPriority(kvrpcpb.CommandPri_Low) - tk.MustExec("alter table t1 add index t1_index (id);") - - c.Assert(atomic.LoadUint32(&cli.lowPriorityCnt) > 0, IsTrue) - - cli.mu.Lock() - cli.mu.checkFlags = checkRequestOff - cli.mu.Unlock() - - tk.MustExec("alter table t1 drop index t1_index;") - tk.MustExec("SET SESSION tidb_ddl_reorg_priority = 'PRIORITY_NORMAL'") - - cli.mu.Lock() - cli.mu.checkFlags = checkDDLAddIndexPriority - cli.mu.Unlock() - - cli.setCheckPriority(kvrpcpb.CommandPri_Normal) - tk.MustExec("alter table t1 add index t1_index (id);") - - cli.mu.Lock() - cli.mu.checkFlags = checkRequestOff - cli.mu.Unlock() - - tk.MustExec("alter table t1 drop index t1_index;") - tk.MustExec("SET SESSION tidb_ddl_reorg_priority = 'PRIORITY_HIGH'") - - cli.mu.Lock() - cli.mu.checkFlags = checkDDLAddIndexPriority - cli.mu.Unlock() - - cli.setCheckPriority(kvrpcpb.CommandPri_High) - tk.MustExec("alter table t1 add index t1_index (id);") - - cli.mu.Lock() - cli.mu.checkFlags = checkRequestOff - cli.mu.Unlock() -} - func (s *testSuite1) TestAlterTableComment(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -3963,7 +3928,7 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "admin check index t c") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[executor:8133]handle 3, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:30, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + c.Assert(err.Error(), Equals, "[admin:8223]data inconsistency in table: t, index: c, handle: 3, index-values:\"handle: 3, values: [KindInt64 30 KindInt64 3]\" != record-values:\"\"") // set data to: // index data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40) @@ -5733,54 +5698,9 @@ func (s *testSerialSuite2) TestUnsignedFeedback(c *C) { c.Assert(result.Rows()[2][6], Equals, "keep order:false") } -func (s *testSuiteWithCliBaseCharset) TestCharsetFeatureWithoutNewCollation(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustQuery("show charset").Check(testkit.Rows( - "ascii US ASCII ascii_bin 1", - "binary binary binary 1", - "gbk Chinese Internal Code Specification gbk_bin 2", - "latin1 Latin1 latin1_bin 1", - "utf8 UTF-8 Unicode utf8_bin 3", - "utf8mb4 UTF-8 Unicode utf8mb4_bin 4", - )) - tk.MustQuery("show collation").Check(testkit.Rows( - "utf8mb4_bin utf8mb4 46 Yes Yes 1", - "latin1_bin latin1 47 Yes Yes 1", - "binary binary 63 Yes Yes 1", - "ascii_bin ascii 65 Yes Yes 1", - "utf8_bin utf8 83 Yes Yes 1", - "gbk_bin gbk 87 Yes Yes 1", - )) - tk.MustExec("create table t5(a char(20), b char(20) charset utf8, c binary) charset gbk collate gbk_bin;") -} - func (s *testSuiteWithCliBaseCharset) TestCharsetFeature(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustQuery("show charset").Check(testkit.Rows( - "ascii US ASCII ascii_bin 1", - "binary binary binary 1", - "gbk Chinese Internal Code Specification gbk_chinese_ci 2", - "latin1 Latin1 latin1_bin 1", - "utf8 UTF-8 Unicode utf8_bin 3", - "utf8mb4 UTF-8 Unicode utf8mb4_bin 4", - )) - tk.MustQuery("show collation").Check(testkit.Rows( - "ascii_bin ascii 65 Yes Yes 1", - "binary binary 63 Yes Yes 1", - "gbk_bin gbk 87 Yes 1", - "gbk_chinese_ci gbk 28 Yes Yes 1", - "latin1_bin latin1 47 Yes Yes 1", - "utf8_bin utf8 83 Yes Yes 1", - "utf8_general_ci utf8 33 Yes 1", - "utf8_unicode_ci utf8 192 Yes 1", - "utf8mb4_bin utf8mb4 46 Yes Yes 1", - "utf8mb4_general_ci utf8mb4 45 Yes 1", - "utf8mb4_unicode_ci utf8mb4 224 Yes 1", - )) tk.MustExec("set names gbk;") tk.MustQuery("select @@character_set_connection;").Check(testkit.Rows("gbk")) @@ -5823,8 +5743,6 @@ func (s *testSuiteWithCliBaseCharset) TestCharsetFeature(c *C) { } func (s *testSuiteWithCliBaseCharset) TestCharsetFeatureCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") @@ -5850,8 +5768,6 @@ func (s *testSuiteWithCliBaseCharset) TestCharsetFeatureCollation(c *C) { } func (s *testSuiteWithCliBaseCharset) TestCharsetWithPrefixIndex(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -9850,6 +9766,30 @@ func (s *testSerialSuite) TestFix31038(c *C) { failpoint.Disable("github.com/pingcap/tidb/store/copr/disable-collect-execution") } +func (s *testSerialSuite) TestFix31530(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk2.MustExec("use test") + defer func() { + tk.MustExec("drop table if exists t1") + }() + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (id int primary key, v int)") + tk.MustExec("insert into t1 values(1, 10)") + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 10")) + + // update t1 before session1 transaction not finished + tk2.MustExec("update t1 set v=11 where id=1") + + tk.MustQuery("(select 'a' as c, id, v from t1 for update) union all (select 'b', id, v from t1) order by c").Check(testkit.Rows("a 1 11", "b 1 10")) + tk.MustQuery("(select 'a' as c, id, v from t1) union all (select 'b', id, v from t1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11")) + tk.MustQuery("(select 'a' as c, id, v from t1 where id=1 for update) union all (select 'b', id, v from t1 where id=1) order by c").Check(testkit.Rows("a 1 11", "b 1 10")) + tk.MustQuery("(select 'a' as c, id, v from t1 where id=1) union all (select 'b', id, v from t1 where id=1 for update) order by c").Check(testkit.Rows("a 1 10", "b 1 11")) + tk.MustExec("rollback") +} + func (s *testSerialSuite) TestFix31537(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -9894,3 +9834,17 @@ func (s *testSerialSuite) TestFix31537(c *C) { );`) tk.MustQuery(`trace plan SELECT T_ID, T_S_SYMB, T_QTY, ST_NAME, TH_DTS FROM ( SELECT T_ID AS ID FROM TRADE WHERE T_CA_ID = 43000014236 ORDER BY T_DTS DESC LIMIT 10 ) T, TRADE, TRADE_HISTORY, STATUS_TYPE WHERE TRADE.T_ID = ID AND TRADE_HISTORY.TH_T_ID = TRADE.T_ID AND STATUS_TYPE.ST_ID = TRADE_HISTORY.TH_ST_ID ORDER BY TH_DTS DESC LIMIT 30;`) } + +func (s *testSerialSuite) TestEncodingSet(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `enum-set` (`set` SET(" + + "'x00','x01','x02','x03','x04','x05','x06','x07','x08','x09','x10','x11','x12','x13','x14','x15'," + + "'x16','x17','x18','x19','x20','x21','x22','x23','x24','x25','x26','x27','x28','x29','x30','x31'," + + "'x32','x33','x34','x35','x36','x37','x38','x39','x40','x41','x42','x43','x44','x45','x46','x47'," + + "'x48','x49','x50','x51','x52','x53','x54','x55','x56','x57','x58','x59','x60','x61','x62','x63'" + + ")NOT NULL PRIMARY KEY)") + tk.MustExec("INSERT INTO `enum-set` VALUES\n(\"x00,x59\");") + tk.MustQuery("select `set` from `enum-set` use index(PRIMARY)").Check(testkit.Rows("x00,x59")) + tk.MustExec("admin check table `enum-set`") +} diff --git a/executor/explain_test.go b/executor/explain_test.go index a26f373e10b71..121f76c00a189 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -18,21 +18,24 @@ import ( "bytes" "fmt" "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser/auth" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestExplainPrivileges(c *C) { - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue) - tk := testkit.NewTestKit(c, s.store) - tk.Se = se +func TestExplainPrivileges(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk := testkit.NewTestKit(t, store) + tk.SetSession(se) tk.MustExec("create database explaindatabase") tk.MustExec("use explaindatabase") @@ -40,11 +43,11 @@ func (s *testSuite1) TestExplainPrivileges(c *C) { tk.MustExec("create view v as select * from t") tk.MustExec(`create user 'explain'@'%'`) - tk1 := testkit.NewTestKit(c, s.store) - se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "explain", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + se, err = session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "explain", Hostname: "%"}, nil, nil)) + tk1.SetSession(se) tk.MustExec(`grant select on explaindatabase.v to 'explain'@'%'`) tk1.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA", "explaindatabase")) @@ -52,7 +55,7 @@ func (s *testSuite1) TestExplainPrivileges(c *C) { tk1.MustExec("use explaindatabase") tk1.MustQuery("select * from v") err = tk1.ExecToErr("explain format = 'brief' select * from v") - c.Assert(err.Error(), Equals, plannercore.ErrViewNoExplain.Error()) + require.Equal(t, plannercore.ErrViewNoExplain.Error(), err.Error()) tk.MustExec(`grant show view on explaindatabase.v to 'explain'@'%'`) tk1.MustQuery("explain format = 'brief' select * from v") @@ -60,11 +63,14 @@ func (s *testSuite1) TestExplainPrivileges(c *C) { tk.MustExec(`revoke select on explaindatabase.v from 'explain'@'%'`) err = tk1.ExecToErr("explain format = 'brief' select * from v") - c.Assert(err.Error(), Equals, plannercore.ErrTableaccessDenied.GenWithStackByArgs("SELECT", "explain", "%", "v").Error()) + require.Equal(t, plannercore.ErrTableaccessDenied.GenWithStackByArgs("SELECT", "explain", "%", "v").Error(), err.Error()) } -func (s *testSuite1) TestExplainCartesianJoin(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainCartesianJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (v int)") @@ -87,12 +93,15 @@ func (s *testSuite1) TestExplainCartesianJoin(c *C) { } } - c.Assert(ok, Equals, ca.isCartesianJoin) + require.Equal(t, ca.isCartesianJoin, ok) } } -func (s *testSuite1) TestExplainWrite(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainWrite(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + 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("explain analyze insert into t select 1") @@ -106,28 +115,31 @@ func (s *testSuite1) TestExplainWrite(c *C) { tk.MustQuery("select * from t order by a").Check(testkit.Rows("1", "2", "3")) } -func (s *testSuite1) TestExplainAnalyzeMemory(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainAnalyzeMemory(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (v int, k int, key(k))") tk.MustExec("insert into t values (1, 1), (1, 1), (1, 1), (1, 1), (1, 1)") - s.checkMemoryInfo(c, tk, "explain analyze select * from t order by v") - s.checkMemoryInfo(c, tk, "explain analyze select * from t order by v limit 5") - s.checkMemoryInfo(c, tk, "explain analyze select /*+ HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.v = t2.v+1") - s.checkMemoryInfo(c, tk, "explain analyze select /*+ MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k+1") - s.checkMemoryInfo(c, tk, "explain analyze select /*+ INL_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkMemoryInfo(c, tk, "explain analyze select /*+ INL_HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkMemoryInfo(c, tk, "explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkMemoryInfo(c, tk, "explain analyze select sum(k) from t group by v") - s.checkMemoryInfo(c, tk, "explain analyze select sum(v) from t group by k") - s.checkMemoryInfo(c, tk, "explain analyze select * from t") - s.checkMemoryInfo(c, tk, "explain analyze select k from t use index(k)") - s.checkMemoryInfo(c, tk, "explain analyze select * from t use index(k)") - s.checkMemoryInfo(c, tk, "explain analyze select v+k from t") + checkMemoryInfo(t, tk, "explain analyze select * from t order by v") + checkMemoryInfo(t, tk, "explain analyze select * from t order by v limit 5") + checkMemoryInfo(t, tk, "explain analyze select /*+ HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.v = t2.v+1") + checkMemoryInfo(t, tk, "explain analyze select /*+ MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k+1") + checkMemoryInfo(t, tk, "explain analyze select /*+ INL_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkMemoryInfo(t, tk, "explain analyze select /*+ INL_HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkMemoryInfo(t, tk, "explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkMemoryInfo(t, tk, "explain analyze select sum(k) from t group by v") + checkMemoryInfo(t, tk, "explain analyze select sum(v) from t group by k") + checkMemoryInfo(t, tk, "explain analyze select * from t") + checkMemoryInfo(t, tk, "explain analyze select k from t use index(k)") + checkMemoryInfo(t, tk, "explain analyze select * from t use index(k)") + checkMemoryInfo(t, tk, "explain analyze select v+k from t") } -func (s *testSuite1) checkMemoryInfo(c *C, tk *testkit.TestKit, sql string) { +func checkMemoryInfo(t *testing.T, tk *testkit.TestKit, sql string) { memCol := 6 ops := []string{"Join", "Reader", "Top", "Sort", "LookUp", "Projection", "Selection", "Agg"} rows := tk.MustQuery(sql).Rows() @@ -149,31 +161,34 @@ func (s *testSuite1) checkMemoryInfo(c *C, tk *testkit.TestKit, sql string) { } if shouldHasMem { - c.Assert(strs[memCol], Not(Equals), "N/A") + require.NotEqual(t, "N/A", strs[memCol]) } else { - c.Assert(strs[memCol], Equals, "N/A") + require.Equal(t, "N/A", strs[memCol]) } } } -func (s *testSuite1) TestMemoryAndDiskUsageAfterClose(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestMemoryAndDiskUsageAfterClose(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (v int, k int, key(k))") batch := 128 - limit := tk.Se.GetSessionVars().MaxChunkSize*2 + 10 + limit := tk.Session().GetSessionVars().MaxChunkSize*2 + 10 var buf bytes.Buffer for i := 0; i < limit; { buf.Reset() _, err := buf.WriteString("insert into t values ") - c.Assert(err, IsNil) + require.NoError(t, err) for j := 0; j < batch && i < limit; i, j = i+1, j+1 { if j > 0 { _, err = buf.WriteString(", ") - c.Assert(err, IsNil) + require.NoError(t, err) } _, err = buf.WriteString(fmt.Sprintf("(%v,%v)", i, i)) - c.Assert(err, IsNil) + require.NoError(t, err) } tk.MustExec(buf.String()) } @@ -185,31 +200,34 @@ func (s *testSuite1) TestMemoryAndDiskUsageAfterClose(c *C) { } for _, sql := range SQLs { tk.MustQuery(sql) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.BytesConsumed(), Equals, int64(0)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) + require.Equal(t, int64(0), tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed()) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), int64(0)) + require.Equal(t, int64(0), tk.Session().GetSessionVars().StmtCtx.DiskTracker.BytesConsumed()) } } -func (s *testSuite2) TestExplainAnalyzeExecutionInfo(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainAnalyzeExecutionInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (v int, k int, key(k))") tk.MustExec("insert into t values (1, 1), (1, 1), (1, 1), (1, 1), (1, 1)") - s.checkExecutionInfo(c, tk, "explain analyze select * from t order by v") - s.checkExecutionInfo(c, tk, "explain analyze select * from t order by v limit 5") - s.checkExecutionInfo(c, tk, "explain analyze select /*+ HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.v = t2.v+1") - s.checkExecutionInfo(c, tk, "explain analyze select /*+ MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k+1") - s.checkExecutionInfo(c, tk, "explain analyze select /*+ INL_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkExecutionInfo(c, tk, "explain analyze select /*+ INL_HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkExecutionInfo(c, tk, "explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") - s.checkExecutionInfo(c, tk, "explain analyze select sum(k) from t group by v") - s.checkExecutionInfo(c, tk, "explain analyze select sum(v) from t group by k") - s.checkExecutionInfo(c, tk, "explain analyze select * from t") - s.checkExecutionInfo(c, tk, "explain analyze select k from t use index(k)") - s.checkExecutionInfo(c, tk, "explain analyze select * from t use index(k)") - s.checkExecutionInfo(c, tk, "explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte;") + checkExecutionInfo(t, tk, "explain analyze select * from t order by v") + checkExecutionInfo(t, tk, "explain analyze select * from t order by v limit 5") + checkExecutionInfo(t, tk, "explain analyze select /*+ HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.v = t2.v+1") + checkExecutionInfo(t, tk, "explain analyze select /*+ MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k+1") + checkExecutionInfo(t, tk, "explain analyze select /*+ INL_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkExecutionInfo(t, tk, "explain analyze select /*+ INL_HASH_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkExecutionInfo(t, tk, "explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ t1.k from t t1, t t2 where t1.k = t2.k and t1.v=1") + checkExecutionInfo(t, tk, "explain analyze select sum(k) from t group by v") + checkExecutionInfo(t, tk, "explain analyze select sum(v) from t group by k") + checkExecutionInfo(t, tk, "explain analyze select * from t") + checkExecutionInfo(t, tk, "explain analyze select k from t use index(k)") + checkExecutionInfo(t, tk, "explain analyze select * from t use index(k)") + checkExecutionInfo(t, tk, "explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte;") tk.MustExec("CREATE TABLE IF NOT EXISTS nation ( N_NATIONKEY BIGINT NOT NULL,N_NAME CHAR(25) NOT NULL,N_REGIONKEY BIGINT NOT NULL,N_COMMENT VARCHAR(152),PRIMARY KEY (N_NATIONKEY));") tk.MustExec("CREATE TABLE IF NOT EXISTS part ( P_PARTKEY BIGINT NOT NULL,P_NAME VARCHAR(55) NOT NULL,P_MFGR CHAR(25) NOT NULL,P_BRAND CHAR(10) NOT NULL,P_TYPE VARCHAR(25) NOT NULL,P_SIZE BIGINT NOT NULL,P_CONTAINER CHAR(10) NOT NULL,P_RETAILPRICE DECIMAL(15,2) NOT NULL,P_COMMENT VARCHAR(23) NOT NULL,PRIMARY KEY (P_PARTKEY));") @@ -218,7 +236,7 @@ func (s *testSuite2) TestExplainAnalyzeExecutionInfo(c *C) { tk.MustExec("CREATE TABLE IF NOT EXISTS orders ( O_ORDERKEY BIGINT NOT NULL,O_CUSTKEY BIGINT NOT NULL,O_ORDERSTATUS CHAR(1) NOT NULL,O_TOTALPRICE DECIMAL(15,2) NOT NULL,O_ORDERDATE DATE NOT NULL,O_ORDERPRIORITY CHAR(15) NOT NULL,O_CLERK CHAR(15) NOT NULL,O_SHIPPRIORITY BIGINT NOT NULL,O_COMMENT VARCHAR(79) NOT NULL,PRIMARY KEY (O_ORDERKEY),CONSTRAINT FOREIGN KEY ORDERS_FK1 (O_CUSTKEY) references customer(C_CUSTKEY));") tk.MustExec("CREATE TABLE IF NOT EXISTS lineitem ( L_ORDERKEY BIGINT NOT NULL,L_PARTKEY BIGINT NOT NULL,L_SUPPKEY BIGINT NOT NULL,L_LINENUMBER BIGINT NOT NULL,L_QUANTITY DECIMAL(15,2) NOT NULL,L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL,L_DISCOUNT DECIMAL(15,2) NOT NULL,L_TAX DECIMAL(15,2) NOT NULL,L_RETURNFLAG CHAR(1) NOT NULL,L_LINESTATUS CHAR(1) NOT NULL,L_SHIPDATE DATE NOT NULL,L_COMMITDATE DATE NOT NULL,L_RECEIPTDATE DATE NOT NULL,L_SHIPINSTRUCT CHAR(25) NOT NULL,L_SHIPMODE CHAR(10) NOT NULL,L_COMMENT VARCHAR(44) NOT NULL,PRIMARY KEY (L_ORDERKEY,L_LINENUMBER),CONSTRAINT FOREIGN KEY LINEITEM_FK1 (L_ORDERKEY) references orders(O_ORDERKEY),CONSTRAINT FOREIGN KEY LINEITEM_FK2 (L_PARTKEY,L_SUPPKEY) references partsupp(PS_PARTKEY, PS_SUPPKEY));") - s.checkExecutionInfo(c, tk, "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%dim%' ) as profit group by nation, o_year order by nation, o_year desc;") + checkExecutionInfo(t, tk, "select nation, o_year, sum(amount) as sum_profit from ( select n_name as nation, extract(year from o_orderdate) as o_year, l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount from part, supplier, lineitem, partsupp, orders, nation where s_suppkey = l_suppkey and ps_suppkey = l_suppkey and ps_partkey = l_partkey and p_partkey = l_partkey and o_orderkey = l_orderkey and s_nationkey = n_nationkey and p_name like '%dim%' ) as profit group by nation, o_year order by nation, o_year desc;") tk.MustExec("drop table if exists nation") tk.MustExec("drop table if exists part") @@ -228,7 +246,7 @@ func (s *testSuite2) TestExplainAnalyzeExecutionInfo(c *C) { tk.MustExec("drop table if exists lineitem") } -func (s *testSuite2) checkExecutionInfo(c *C, tk *testkit.TestKit, sql string) { +func checkExecutionInfo(t *testing.T, tk *testkit.TestKit, sql string) { executionInfoCol := 4 rows := tk.MustQuery(sql).Rows() for _, row := range rows { @@ -236,21 +254,23 @@ func (s *testSuite2) checkExecutionInfo(c *C, tk *testkit.TestKit, sql string) { for i, c := range row { strs[i] = c.(string) } - - c.Assert(strs[executionInfoCol], Not(Equals), "time:0s, loops:0, rows:0") + require.NotEqual(t, "time:0s, loops:0, rows:0", strs[executionInfoCol]) } } -func (s *testSuite2) TestExplainAnalyzeActRowsNotEmpty(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainAnalyzeActRowsNotEmpty(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, index (a))") tk.MustExec("insert into t values (1, 1)") - s.checkActRowsNotEmpty(c, tk, "explain analyze select * from t t1, t t2 where t1.b = t2.a and t1.b = 2333") + checkActRowsNotEmpty(t, tk, "explain analyze select * from t t1, t t2 where t1.b = t2.a and t1.b = 2333") } -func (s *testSuite2) checkActRowsNotEmpty(c *C, tk *testkit.TestKit, sql string) { +func checkActRowsNotEmpty(t *testing.T, tk *testkit.TestKit, sql string) { actRowsCol := 2 rows := tk.MustQuery(sql).Rows() for _, row := range rows { @@ -258,28 +278,30 @@ func (s *testSuite2) checkActRowsNotEmpty(c *C, tk *testkit.TestKit, sql string) for i, c := range row { strs[i] = c.(string) } - - c.Assert(strs[actRowsCol], Not(Equals), "") + require.NotEqual(t, "", strs[actRowsCol]) } } -func checkActRows(c *C, tk *testkit.TestKit, sql string, expected []string) { +func checkActRows(t *testing.T, tk *testkit.TestKit, sql string, expected []string) { actRowsCol := 2 rows := tk.MustQuery("explain analyze " + sql).Rows() - c.Assert(len(rows), Equals, len(expected)) + require.Equal(t, len(expected), len(rows)) for id, row := range rows { strs := make([]string, len(row)) for i, c := range row { strs[i] = c.(string) } - c.Assert(strs[actRowsCol], Equals, expected[id]) + require.Equal(t, expected[id], strs[actRowsCol], fmt.Sprintf("error comparing %s", sql)) } } -func (s *testSuite1) TestCheckActRowsWithUnistore(c *C) { +func TestCheckActRowsWithUnistore(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // testSuite1 use default mockstore which is unistore - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t_unistore_act_rows") tk.MustExec("create table t_unistore_act_rows(a int, b int, index(a, b))") tk.MustExec("insert into t_unistore_act_rows values (1, 0), (1, 0), (2, 0), (2, 1)") @@ -330,12 +352,15 @@ func (s *testSuite1) TestCheckActRowsWithUnistore(c *C) { } for _, test := range tests { - checkActRows(c, tk, test.sql, test.expected) + checkActRows(t, tk, test.sql, test.expected) } } -func (s *testSuite2) TestExplainAnalyzeCTEMemoryAndDiskInfo(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainAnalyzeCTEMemoryAndDiskInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int)") tk.MustExec("insert into t with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte;") @@ -343,19 +368,22 @@ func (s *testSuite2) TestExplainAnalyzeCTEMemoryAndDiskInfo(c *C) { rows := tk.MustQuery("explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000)" + " select * from cte, t;").Rows() - c.Assert(rows[4][7].(string), Not(Equals), "N/A") - c.Assert(rows[4][8].(string), Equals, "0 Bytes") + require.NotEqual(t, "N/A", rows[4][7].(string)) + require.Equal(t, "0 Bytes", rows[4][8].(string)) tk.MustExec("set @@tidb_mem_quota_query=10240;") rows = tk.MustQuery("explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000)" + " select * from cte, t;").Rows() - c.Assert(rows[4][7].(string), Not(Equals), "N/A") - c.Assert(rows[4][8].(string), Not(Equals), "N/A") + require.NotEqual(t, "N/A", rows[4][7].(string)) + require.NotEqual(t, "N/A", rows[4][8].(string)) } -func (s *testSuite) TestExplainStatementsSummary(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainStatementsSummary(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustQuery("desc select * from information_schema.statements_summary").Check(testkit.Rows( `MemTableScan_4 10000.00 root table:STATEMENTS_SUMMARY `)) tk.MustQuery("desc select * from information_schema.statements_summary where digest is null").Check(testutil.RowsWithSep("|", @@ -366,8 +394,11 @@ func (s *testSuite) TestExplainStatementsSummary(c *C) { `MemTableScan_5 10000.00 root table:STATEMENTS_SUMMARY digests: ["a","b","c"]`)) } -func (s *testSuite) TestFix29401(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFix29401(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists tt123;") tk.MustExec(`CREATE TABLE tt123 ( id int(11) NOT NULL, diff --git a/executor/hash_table.go b/executor/hash_table.go index b22f98bbef501..5476c5901e4f1 100644 --- a/executor/hash_table.go +++ b/executor/hash_table.go @@ -143,6 +143,7 @@ func (c *hashRowContainer) matchJoinKey(buildRow, probeRow chunk.Row, probeHCtx } // alreadySpilledSafeForTest indicates that records have spilled out into disk. It's thread-safe. +// nolint: unused func (c *hashRowContainer) alreadySpilledSafeForTest() bool { return c.rowContainer.AlreadySpilledSafeForTest() } diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 400a2b1d91638..69ffe2265905b 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -120,29 +120,7 @@ type indexHashJoinTask struct { // Open implements the IndexNestedLoopHashJoin Executor interface. func (e *IndexNestedLoopHashJoin) Open(ctx context.Context) error { - // Be careful, very dirty hack in this line!!! - // IndexLookUpJoin need to rebuild executor (the dataReaderBuilder) during - // executing. However `executor.Next()` is lazy evaluation when the RecordSet - // result is drained. - // Lazy evaluation means the saved session context may change during executor's - // building and its running. - // A specific sequence for example: - // - // e := buildExecutor() // txn at build time - // recordSet := runStmt(e) - // session.CommitTxn() // txn closed - // recordSet.Next() - // e.dataReaderBuilder.Build() // txn is used again, which is already closed - // - // The trick here is `getSnapshotTS` will cache snapshot ts in the dataReaderBuilder, - // so even txn is destroyed later, the dataReaderBuilder could still use the - // cached snapshot ts to construct DAG. - _, err := e.innerCtx.readerBuilder.getSnapshotTS() - if err != nil { - return err - } - - err = e.children[0].Open(ctx) + err := e.children[0].Open(ctx) if err != nil { return err } @@ -676,7 +654,7 @@ func (iw *indexHashJoinInnerWorker) doJoinUnordered(ctx context.Context, task *i for row := iter.Begin(); row != iter.End(); row = iter.Next() { ok, joinResult = iw.joinMatchedInnerRow2Chunk(ctx, row, task, joinResult, h, iw.joinKeyBuf) if !ok { - return errors.New("indexHashJoinInnerWorker.doJoinUnordered failed") + return joinResult.err } } for chkIdx, outerRowStatus := range task.outerRowStatus { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index a4c6c8c01b575..44a7bdb4b9ca1 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -161,29 +161,7 @@ type innerWorker struct { // Open implements the Executor interface. func (e *IndexLookUpJoin) Open(ctx context.Context) error { - // Be careful, very dirty hack in this line!!! - // IndexLookUpJoin need to rebuild executor (the dataReaderBuilder) during - // executing. However `executor.Next()` is lazy evaluation when the RecordSet - // result is drained. - // Lazy evaluation means the saved session context may change during executor's - // building and its running. - // A specific sequence for example: - // - // e := buildExecutor() // txn at build time - // recordSet := runStmt(e) - // session.CommitTxn() // txn closed - // recordSet.Next() - // e.dataReaderBuilder.Build() // txn is used again, which is already closed - // - // The trick here is `getSnapshotTS` will cache snapshot ts in the dataReaderBuilder, - // so even txn is destroyed later, the dataReaderBuilder could still use the - // cached snapshot ts to construct DAG. - _, err := e.innerCtx.readerBuilder.getSnapshotTS() - if err != nil { - return err - } - - err = e.children[0].Open(ctx) + err := e.children[0].Open(ctx) if err != nil { return err } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 7e4ac6a515ae9..db1c27a5dfdf7 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -156,29 +156,7 @@ type indexMergeJoinResult struct { // Open implements the Executor interface func (e *IndexLookUpMergeJoin) Open(ctx context.Context) error { - // Be careful, very dirty hack in this line!!! - // IndexLookMergeUpJoin need to rebuild executor (the dataReaderBuilder) during - // executing. However `executor.Next()` is lazy evaluation when the RecordSet - // result is drained. - // Lazy evaluation means the saved session context may change during executor's - // building and its running. - // A specific sequence for example: - // - // e := buildExecutor() // txn at build time - // recordSet := runStmt(e) - // session.CommitTxn() // txn closed - // recordSet.Next() - // e.dataReaderBuilder.Build() // txn is used again, which is already closed - // - // The trick here is `getSnapshotTS` will cache snapshot ts in the dataReaderBuilder, - // so even txn is destroyed later, the dataReaderBuilder could still use the - // cached snapshot ts to construct DAG. - _, err := e.innerMergeCtx.readerBuilder.getSnapshotTS() - if err != nil { - return err - } - - err = e.children[0].Open(ctx) + err := e.children[0].Open(ctx) if err != nil { return err } diff --git a/executor/index_lookup_merge_join_test.go b/executor/index_lookup_merge_join_test.go index 16b74c00d39b4..44bdf29cb8598 100644 --- a/executor/index_lookup_merge_join_test.go +++ b/executor/index_lookup_merge_join_test.go @@ -16,21 +16,25 @@ package executor_test import ( "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/plancodec" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSerialSuite) TestIndexLookupMergeJoinHang(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/IndexMergeJoinMockOOM", `return(true)`), IsNil) +func TestIndexLookupMergeJoinHang(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/IndexMergeJoinMockOOM", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/IndexMergeJoinMockOOM"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/IndexMergeJoinMockOOM")) }() - - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (a int,b int,index idx(a))") tk.MustExec("create table t2 (a int,b int,index idx(a))") @@ -38,12 +42,14 @@ func (s *testSerialSuite) TestIndexLookupMergeJoinHang(c *C) { tk.MustExec("insert into t2 values (1,1),(2,2),(3,3),(2000,2000)") // Do not hang in index merge join when OOM occurs. err := tk.QueryToErr("select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1, t2 where t1.a = t2.a") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "OOM test index merge join doesn't hang here.") + require.Error(t, err) + require.Equal(t, "OOM test index merge join doesn't hang here.", err.Error()) } -func (s *testSerialSuite) TestIssue28052(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue28052(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -58,13 +64,15 @@ func (s *testSerialSuite) TestIssue28052(c *C) { tk.MustQuery("select /*+ inl_merge_join(t1, t2) */ count(*) from t t1 right join t t2 on t1. `col_year_key_signed` = t2. `col_tinyint_key_signed`").Check(testkit.Rows("1")) } -func (s *testSerialSuite) TestIssue18068(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testIssue18068", `return(true)`), IsNil) +func TestIssue18068(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIssue18068", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testIssue18068"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIssue18068")) }() - - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t, s") tk.MustExec("create table t (a int, index idx(a))") tk.MustExec("create table s (a int, index idx(a))") @@ -81,23 +89,29 @@ func (s *testSerialSuite) TestIssue18068(c *C) { tk.MustExec("select /*+ inl_merge_join(s)*/ 1 from t join s on t.a = s.a limit 1") } -func (s *testSuite9) TestIssue18631(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue18631(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, d int, primary key(a,b,c))") tk.MustExec("create table t2(a int, b int, c int, d int, primary key(a,b,c))") tk.MustExec("insert into t1 values(1,1,1,1),(2,2,2,2),(3,3,3,3)") tk.MustExec("insert into t2 values(1,1,1,1),(2,2,2,2)") firstOperator := tk.MustQuery("explain format = 'brief' select /*+ inl_merge_join(t1,t2) */ * from t1 left join t2 on t1.a = t2.a and t1.c = t2.c and t1.b = t2.b order by t1.a desc").Rows()[0][0].(string) - c.Assert(strings.Index(firstOperator, plancodec.TypeIndexMergeJoin), Equals, 0) + require.Equal(t, 0, strings.Index(firstOperator, plancodec.TypeIndexMergeJoin)) tk.MustQuery("select /*+ inl_merge_join(t1,t2) */ * from t1 left join t2 on t1.a = t2.a and t1.c = t2.c and t1.b = t2.b order by t1.a desc").Check(testkit.Rows( "3 3 3 3 ", "2 2 2 2 2 2 2 2", "1 1 1 1 1 1 1 1")) } -func (s *testSuite9) TestIssue19408(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue19408(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (c_int int, primary key(c_int))") tk.MustExec("create table t2 (c_int int, unique key (c_int)) partition by hash (c_int) partitions 4") @@ -123,8 +137,11 @@ func (s *testSuite9) TestIssue19408(c *C) { tk.MustExec("commit") } -func (s *testSuite9) TestIssue20137(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20137(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (id bigint(20) unsigned, primary key(id))") tk.MustExec("create table t2 (id bigint(20) unsigned)") @@ -136,9 +153,12 @@ func (s *testSuite9) TestIssue20137(c *C) { testkit.Rows("8738875760185212610 8738875760185212610", "9814441339970117597 9814441339970117597")) } -func (s *testSuiteWithData) TestIndexJoinOnSinglePartitionTable(c *C) { +func TestIndexJoinOnSinglePartitionTable(t *testing.T) { // For issue 19145 - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { tk.MustExec("set @@tidb_partition_prune_mode= '" + val + "'") tk.MustExec("drop table if exists t1, t2") @@ -148,25 +168,28 @@ func (s *testSuiteWithData) TestIndexJoinOnSinglePartitionTable(c *C) { tk.MustExec("insert into t2 values (1, 'Bob')") sql := "select /*+ INL_MERGE_JOIN(t1,t2) */ * from t1 join t2 partition(p0) on t1.c_int = t2.c_int and t1.c_str < t2.c_str" tk.MustQuery(sql).Check(testkit.Rows("1 Alice 1 Bob")) - rows := s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) + rows := testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) // Partition table can't be inner side of index merge join, because it can't keep order. - c.Assert(strings.Index(rows[0], "IndexMergeJoin"), Equals, -1) - c.Assert(len(tk.MustQuery("show warnings").Rows()) > 0, Equals, true) + require.Equal(t, -1, strings.Index(rows[0], "IndexMergeJoin")) + require.Equal(t, true, len(tk.MustQuery("show warnings").Rows()) > 0) sql = "select /*+ INL_HASH_JOIN(t1,t2) */ * from t1 join t2 partition(p0) on t1.c_int = t2.c_int and t1.c_str < t2.c_str" tk.MustQuery(sql).Check(testkit.Rows("1 Alice 1 Bob")) - rows = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) - c.Assert(strings.Index(rows[0], "IndexHashJoin"), Equals, 0) + rows = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) + require.Equal(t, 0, strings.Index(rows[0], "IndexHashJoin")) sql = "select /*+ INL_JOIN(t1,t2) */ * from t1 join t2 partition(p0) on t1.c_int = t2.c_int and t1.c_str < t2.c_str" tk.MustQuery(sql).Check(testkit.Rows("1 Alice 1 Bob")) - rows = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) - c.Assert(strings.Index(rows[0], "IndexJoin"), Equals, 0) + rows = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + sql).Rows()) + require.Equal(t, 0, strings.Index(rows[0], "IndexJoin")) } } -func (s *testSuite9) TestIssue20400(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20400(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t, s") tk.MustExec("create table s(a int, index(a))") tk.MustExec("create table t(a int)") @@ -177,8 +200,11 @@ func (s *testSuite9) TestIssue20400(c *C) { testkit.Rows("1 ")) } -func (s *testSuite9) TestIssue20549(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20549(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("CREATE TABLE `t1` (`id` bigint(20) NOT NULL AUTO_INCREMENT, `t2id` bigint(20) DEFAULT NULL, PRIMARY KEY (`id`), KEY `t2id` (`t2id`));") tk.MustExec("INSERT INTO `t1` VALUES (1,NULL);") @@ -189,8 +215,11 @@ func (s *testSuite9) TestIssue20549(c *C) { testkit.Rows("1")) } -func (s *testSuite9) TestIssue24473AndIssue25669(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue24473AndIssue25669(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists x, t2, t3") tk.MustExec("CREATE TABLE `x` ( `a` enum('y','b','1','x','0','null') DEFAULT NULL, KEY `a` (`a`));") tk.MustExec("insert into x values(\"x\"),(\"x\"),(\"b\"),(\"y\");") diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 38aa4a84f5bc5..66dde5b0e743a 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -118,6 +118,11 @@ type IndexMergeReaderExecutor struct { isCorColInPartialAccess []bool } +// Table implements the dataSourceExecutor interface. +func (e *IndexMergeReaderExecutor) Table() table.Table { + return e.table +} + // Open implements the Executor Open interface func (e *IndexMergeReaderExecutor) Open(ctx context.Context) (err error) { e.keyRanges = make([][]kv.KeyRange, 0, len(e.partialPlans)) @@ -296,9 +301,6 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, } var builder distsql.RequestBuilder - if e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - builder.SetIsolationLevel(kv.RC) - } builder.SetDAGRequest(e.dagPBs[workID]). SetStartTS(e.startTS). SetDesc(e.descs[workID]). diff --git a/executor/index_merge_reader_test.go b/executor/index_merge_reader_test.go index 99670af4807f7..ad57e3d050b40 100644 --- a/executor/index_merge_reader_test.go +++ b/executor/index_merge_reader_test.go @@ -20,15 +20,18 @@ import ( "regexp" "strconv" "strings" + "testing" - . "github.com/pingcap/check" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/israce" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestSingleTableRead(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestSingleTableRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(id int primary key, a int, b int, c int, d int)") tk.MustExec("create index t1a on t1(a)") @@ -46,8 +49,11 @@ func (s *testSuite1) TestSingleTableRead(c *C) { tk.MustQuery("select /*+ use_index_merge(t1, t1a, t1b) */ sum(a) from t1 where a < 2 or b > 4").Check(testkit.Rows("6")) } -func (s *testSuite1) TestJoin(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(id int primary key, a int, b int, c int, d int)") tk.MustExec("create index t1a on t1(a)") @@ -60,8 +66,10 @@ func (s *testSuite1) TestJoin(c *C) { tk.MustQuery("select /*+ use_index_merge(t1, t1a, t1b) */ sum(t1.a) from t1 join t2 on t1.id = t2.id where t1.a < 2 or t1.b > 5").Check(testkit.Rows("1")) } -func (s *testSuite1) TestIndexMergeReaderAndGeneratedColumn(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIndexMergeReaderAndGeneratedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT AS (1), c1 INT PRIMARY KEY)") @@ -72,8 +80,10 @@ func (s *testSuite1) TestIndexMergeReaderAndGeneratedColumn(c *C) { } // issue 25045 -func (s *testSuite1) TestIndexMergeReaderIssue25045(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIndexMergeReaderIssue25045(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(a int primary key, b int, c int, key(b), key(c));") @@ -87,8 +97,11 @@ func (s *testSuite1) TestIndexMergeReaderIssue25045(c *C) { tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where c=10 or (b=10 and a=10);").Check(testkit.Rows("10 10 10")) } -func (s *testSuite1) TestIssue16910(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue16910(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("use test;") tk.MustExec("drop table if exists t1, t2, t3;") tk.MustExec("create table t1 (a int not null, b tinyint not null, index (a), index (b)) partition by range (a) (" + @@ -104,19 +117,20 @@ func (s *testSuite1) TestIssue16910(c *C) { tk.MustQuery("select /*+ USE_INDEX_MERGE(t1, a, b) */ * from t1 partition (p0) join t2 partition (p1) on t1.a = t2.a where t1.a < 40 or t1.b < 30;").Check(testkit.Rows("1 1 1 1")) } -func (s *testSuite1) TestIndexMergeCausePanic(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeCausePanic(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_index_merge = 1;") tk.MustExec("create table t (a int, b int, c int, primary key(a), key(b))") tk.MustQuery("explain select /*+ inl_join(t2) */ * from t t1 join t t2 on t1.a = t2.a and t1.c = t2.c where t2.a = 1 or t2.b = 1") } -func (s *testSuite1) TestPartitionTableRandomIndexMerge(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestPartitionTableRandomIndexMerge(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_index_merge=1") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") @@ -174,8 +188,10 @@ func (s *testSuite1) TestPartitionTableRandomIndexMerge(c *C) { } } -func (s *testSuite1) TestIndexMergeWithPreparedStmt(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeWithPreparedStmt(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 int, c2 int, c3 int, key(c1), key(c2));") @@ -188,26 +204,29 @@ func (s *testSuite1) TestIndexMergeWithPreparedStmt(c *C) { tk.MustExec("prepare stmt1 from 'select /*+ use_index_merge(t1) */ count(1) from t1 where c1 < ? or c2 < ?';") tk.MustExec("set @a = 10;") tk.MustQuery("execute stmt1 using @a, @a;").Check(testkit.Rows("10")) - tk.Se.SetSessionManager(&mockSessionManager1{ - PS: []*util.ProcessInfo{tk.Se.ShowProcess()}, + tk.Session().SetSessionManager(&mockSessionManager1{ + PS: []*util.ProcessInfo{tk.Session().ShowProcess()}, }) - explainStr := "explain for connection " + strconv.FormatUint(tk.Se.ShowProcess().ID, 10) + explainStr := "explain for connection " + strconv.FormatUint(tk.Session().ShowProcess().ID, 10) res := tk.MustQuery(explainStr) indexMergeLine := res.Rows()[1][0].(string) re, err := regexp.Compile(".*IndexMerge.*") - c.Assert(err, IsNil) - c.Assert(re.MatchString(indexMergeLine), IsTrue) + require.NoError(t, err) + require.True(t, re.MatchString(indexMergeLine)) tk.MustExec("prepare stmt1 from 'select /*+ use_index_merge(t1) */ count(1) from t1 where c1 < ? or c2 < ? and c3';") tk.MustExec("set @a = 10;") tk.MustQuery("execute stmt1 using @a, @a;").Check(testkit.Rows("10")) res = tk.MustQuery(explainStr) indexMergeLine = res.Rows()[1][0].(string) - c.Assert(re.MatchString(indexMergeLine), IsTrue) + require.True(t, re.MatchString(indexMergeLine)) } -func (s *testSuite1) TestIndexMergeInTransaction(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIndexMergeInTransaction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") for i := 0; i < 2; i++ { tk.MustExec("drop table if exists t1;") @@ -303,7 +322,7 @@ func (s *testSuite1) TestIndexMergeInTransaction(c *C) { // Test partition table. tk.MustExec("drop table if exists t1;") - tk.MustExec(`create table t1(c1 int, c2 int, c3 int, pk int, part int, key(c1), key(c2), key(c3), primary key(pk, part)) + tk.MustExec(`create table t1(c1 int, c2 int, c3 int, pk int, part int, key(c1), key(c2), key(c3), primary key(pk, part)) partition by range(part) ( partition p0 values less than (10), partition p1 values less than (20), @@ -350,15 +369,18 @@ func (s *testSuite1) TestIndexMergeInTransaction(c *C) { tk.MustExec("commit;") } -func (s *testSuite1) TestIndexMergeReaderInTransIssue30685(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIndexMergeReaderInTransIssue30685(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") // This is a case generated by sqlgen to test if clustered index is ok. // Detect the bugs in memIndexMergeReader.getMemRowsHandle(). tk.MustExec("drop table if exists t1;") tk.MustExec(`create table t1 (col_30 decimal default 0 , - col_31 char(99) collate utf8_bin default 'sVgzHblmYYtEjVg' not null , - col_37 int unsigned default 377206828 , + col_31 char(99) collate utf8_bin default 'sVgzHblmYYtEjVg' not null , + col_37 int unsigned default 377206828 , primary key idx_16 ( col_37 ) , key idx_19 ( col_31) ) collate utf8mb4_general_ci ;`) tk.MustExec("begin;") tk.MustExec("insert ignore into t1 values (388021, '', 416235653);") @@ -367,14 +389,14 @@ func (s *testSuite1) TestIndexMergeReaderInTransIssue30685(c *C) { tk.MustExec("drop table if exists tbl_3;") tk.MustExec(`create table tbl_3 ( col_30 decimal , col_31 char(99) , col_32 smallint , - col_33 tinyint unsigned not null , col_34 char(209) , + col_33 tinyint unsigned not null , col_34 char(209) , col_35 char(110) , col_36 int unsigned , col_37 int unsigned , - col_38 decimal(50,15) not null , col_39 char(104), + col_38 decimal(50,15) not null , col_39 char(104), primary key ( col_37 ) , unique key ( col_33,col_30,col_36,col_39 ) , - unique key ( col_32,col_35 ) , key ( col_31,col_38 ) , - key ( col_31,col_33,col_32,col_35,col_36 ) , - unique key ( col_38,col_34,col_33,col_31,col_30,col_36,col_35,col_37,col_39 ) , - unique key ( col_39,col_32 ) , unique key ( col_30,col_35,col_31,col_38 ) , + unique key ( col_32,col_35 ) , key ( col_31,col_38 ) , + key ( col_31,col_33,col_32,col_35,col_36 ) , + unique key ( col_38,col_34,col_33,col_31,col_30,col_36,col_35,col_37,col_39 ) , + unique key ( col_39,col_32 ) , unique key ( col_30,col_35,col_31,col_38 ) , key ( col_38,col_32,col_33 ) )`) tk.MustExec("begin;") tk.MustExec("insert ignore into tbl_3 values ( 71,'Fipc',-6676,30,'','FgfK',2464927398,4084082400,5602.5868,'' );") @@ -420,8 +442,10 @@ func (s *testSuite1) TestIndexMergeReaderInTransIssue30685(c *C) { tk.MustExec("commit;") } -func (test *testSerialSuite2) TestIndexMergeReaderMemTracker(c *C) { - tk := testkit.NewTestKit(c, test.store) +func TestIndexMergeReaderMemTracker(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("create table t1(c1 int, c2 int, c3 int, key(c1), key(c2), key(c3));") @@ -431,7 +455,7 @@ func (test *testSerialSuite2) TestIndexMergeReaderMemTracker(c *C) { insertStr += fmt.Sprintf(" ,(%d, %d, %d)", i, i, i) } insertStr += ";" - memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker + memTracker := tk.Session().GetSessionVars().StmtCtx.MemTracker tk.MustExec(insertStr) @@ -441,22 +465,24 @@ func (test *testSerialSuite2) TestIndexMergeReaderMemTracker(c *C) { tk.MustQuery("select /*+ use_index_merge(t1) */ * from t1 where c1 > 1 or c2 > 1") newMaxUsage := memTracker.MaxConsumed() - c.Assert(newMaxUsage, Greater, oriMaxUsage) + require.Greater(t, newMaxUsage, oriMaxUsage) res := tk.MustQuery("explain analyze select /*+ use_index_merge(t1) */ * from t1 where c1 > 1 or c2 > 1") - c.Assert(len(res.Rows()), Equals, 4) + require.Len(t, res.Rows(), 4) // Parse "xxx KB" and check it's greater than 0. memStr := res.Rows()[0][7].(string) re, err := regexp.Compile("[0-9]+ KB") - c.Assert(err, IsNil) - c.Assert(re.MatchString(memStr), IsTrue) + require.NoError(t, err) + require.True(t, re.MatchString(memStr)) bytes, err := strconv.ParseFloat(memStr[:len(memStr)-3], 32) - c.Assert(err, IsNil) - c.Assert(bytes, Greater, 0.0) + require.NoError(t, err) + require.Greater(t, bytes, 0.0) } -func (test *testSerialSuite2) TestIndexMergeSplitTable(c *C) { - tk := testkit.NewTestKit(c, test.store) +func TestIndexMergeSplitTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("DROP TABLE IF EXISTS tab2;") tk.MustExec("CREATE TABLE tab2(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT);") diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index efbe677109c55..42ee1b155e6e9 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -141,8 +141,6 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex e.setDataFromUserPrivileges(sctx) case infoschema.TableTiKVRegionStatus: err = e.setDataForTiKVRegionStatus(sctx) - case infoschema.TableTiKVRegionPeers: - err = e.setDataForTikVRegionPeers(sctx) case infoschema.TableTiDBHotRegions: err = e.setDataForTiDBHotRegions(sctx) case infoschema.TableConstraints: @@ -190,11 +188,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex func getRowCountAllTable(ctx context.Context, sctx sessionctx.Context) (map[int64]uint64, error) { exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "select table_id, count from mysql.stats_meta") - if err != nil { - return nil, err - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select table_id, count from mysql.stats_meta") if err != nil { return nil, err } @@ -215,11 +209,7 @@ type tableHistID struct { func getColLengthAllTables(ctx context.Context, sctx sessionctx.Context) (map[tableHistID]uint64, error) { exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0") - if err != nil { - return nil, err - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select table_id, hist_id, tot_col_size from mysql.stats_histograms where is_index = 0") if err != nil { return nil, err } @@ -1201,7 +1191,7 @@ func (e *DDLJobsReaderExec) Next(ctx context.Context, req *chunk.Chunk) error { num := mathutil.Min(req.Capacity(), len(e.runningJobs)-e.cursor) for i := e.cursor; i < e.cursor+num; i++ { e.appendJobToChunk(req, e.runningJobs[i], checker) - req.AppendString(11, e.runningJobs[i].Query) + req.AppendString(12, e.runningJobs[i].Query) } e.cursor += num count += num @@ -1216,7 +1206,7 @@ func (e *DDLJobsReaderExec) Next(ctx context.Context, req *chunk.Chunk) error { } for _, job := range e.cacheJobs { e.appendJobToChunk(req, job, checker) - req.AppendString(11, job.Query) + req.AppendString(12, job.Query) } e.cursor += len(e.cacheJobs) } @@ -1536,63 +1526,6 @@ func (e *memtableRetriever) setNewTiKVRegionStatusCol(region *helper.RegionInfo, e.rows = append(e.rows, row) } -func (e *memtableRetriever) setDataForTikVRegionPeers(ctx sessionctx.Context) error { - tikvStore, ok := ctx.GetStore().(helper.Storage) - if !ok { - return errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") - } - tikvHelper := &helper.Helper{ - Store: tikvStore, - RegionCache: tikvStore.GetRegionCache(), - } - regionsInfo, err := tikvHelper.GetRegionsInfo() - if err != nil { - return err - } - for i := range regionsInfo.Regions { - e.setNewTiKVRegionPeersCols(®ionsInfo.Regions[i]) - } - return nil -} - -func (e *memtableRetriever) setNewTiKVRegionPeersCols(region *helper.RegionInfo) { - records := make([][]types.Datum, 0, len(region.Peers)) - pendingPeerIDSet := set.NewInt64Set() - for _, peer := range region.PendingPeers { - pendingPeerIDSet.Insert(peer.ID) - } - downPeerMap := make(map[int64]int64, len(region.DownPeers)) - for _, peerStat := range region.DownPeers { - downPeerMap[peerStat.Peer.ID] = peerStat.DownSec - } - for _, peer := range region.Peers { - row := make([]types.Datum, len(infoschema.TableTiKVRegionPeersCols)) - row[0].SetInt64(region.ID) - row[1].SetInt64(peer.ID) - row[2].SetInt64(peer.StoreID) - if peer.IsLearner { - row[3].SetInt64(1) - } else { - row[3].SetInt64(0) - } - if peer.ID == region.Leader.ID { - row[4].SetInt64(1) - } else { - row[4].SetInt64(0) - } - if downSec, ok := downPeerMap[peer.ID]; ok { - row[5].SetString(downPeer, mysql.DefaultCollationName) - row[6].SetInt64(downSec) - } else if pendingPeerIDSet.Exist(peer.ID) { - row[5].SetString(pendingPeer, mysql.DefaultCollationName) - } else { - row[5].SetString(normalPeer, mysql.DefaultCollationName) - } - records = append(records, row) - } - e.rows = append(e.rows, records...) -} - const ( normalPeer = "NORMAL" pendingPeer = "PENDING" @@ -2144,7 +2077,7 @@ func (e *stmtSummaryTableRetriever) retrieve(ctx context.Context, sctx sessionct } } user := sctx.GetSessionVars().User - reader := stmtsummary.NewStmtSummaryReader(user, hasPriv(sctx, mysql.ProcessPriv), e.columns, instanceAddr) + reader := stmtsummary.NewStmtSummaryReader(user, hasPriv(sctx, mysql.ProcessPriv), e.columns, instanceAddr, sctx.GetSessionVars().StmtCtx.TimeZone) if e.extractor.Enable { checker := stmtsummary.NewStmtSummaryChecker(e.extractor.Digests) reader.SetChecker(checker) @@ -2892,6 +2825,12 @@ func (e *memtableRetriever) setDataFromPlacementPolicies(sctx sessionctx.Context // also convert them to LeaderConstraints and FollowerConstraints // for better user experience searching for particular constraints + // Followers == 0 means not set, so the default value 2 will be used + followerCnt := policy.PlacementSettings.Followers + if followerCnt == 0 { + followerCnt = 2 + } + row := types.MakeDatums( policy.ID, infoschema.CatalogVal, // CATALOG @@ -2903,7 +2842,7 @@ func (e *memtableRetriever) setDataFromPlacementPolicies(sctx sessionctx.Context policy.PlacementSettings.FollowerConstraints, policy.PlacementSettings.LearnerConstraints, policy.PlacementSettings.Schedule, - policy.PlacementSettings.Followers, + followerCnt, policy.PlacementSettings.Learners, ) rows = append(rows, row) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 1febaaaa8d90d..d19484f3691c7 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -218,22 +218,6 @@ func (s *testInfoschemaTableSuite) TestEngines(c *C) { tk.MustQuery("select * from information_schema.ENGINES;").Check(testkit.Rows("InnoDB DEFAULT Supports transactions, row-level locking, and foreign keys YES YES YES")) } -func (s *testInfoschemaTableSuite) TestCharacterSetCollations(c *C) { - tk := testkit.NewTestKit(c, s.store) - - // The description column is not important - tk.MustQuery("SELECT default_collate_name, maxlen FROM information_schema.character_sets ORDER BY character_set_name").Check( - testkit.Rows("ascii_bin 1", "binary 1", "gbk_bin 2", "latin1_bin 1", "utf8_bin 3", "utf8mb4_bin 4")) - - // The is_default column is not important - // but the id's are used by client libraries and must be stable - tk.MustQuery("SELECT character_set_name, id, sortlen FROM information_schema.collations ORDER BY collation_name").Check( - testkit.Rows("ascii 65 1", "binary 63 1", "gbk 87 1", "latin1 47 1", "utf8 83 1", "utf8mb4 46 1")) - - tk.MustQuery("select * from information_schema.COLLATION_CHARACTER_SET_APPLICABILITY where COLLATION_NAME='utf8mb4_bin';").Check( - testkit.Rows("utf8mb4_bin utf8mb4")) -} - // https://github.com/pingcap/tidb/issues/25467. func (s *testInfoschemaTableSuite) TestDataTypesMaxLengthAndOctLength(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/insert_test.go b/executor/insert_test.go index c72fd6134401d..93f0f06763b08 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -19,65 +19,66 @@ import ( "math" "strconv" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/israce" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertOnDuplicateKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(a1 bigint primary key, b1 bigint);`) tk.MustExec(`create table t2(a2 bigint primary key, b2 bigint);`) tk.MustExec(`insert into t1 values(1, 100);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t2 values(1, 200);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 select a2, b2 from t2 on duplicate key update b1 = a2;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 1")) tk.MustExec(`insert into t1 select a2, b2 from t2 on duplicate key update b1 = b2;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 200")) tk.MustExec(`insert into t1 select a2, b2 from t2 on duplicate key update a1 = a2;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 200")) tk.MustExec(`insert into t1 select a2, b2 from t2 on duplicate key update b1 = 300;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 300")) tk.MustExec(`insert into t1 values(1, 1) on duplicate key update b1 = 400;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 400")) tk.MustExec(`insert into t1 select 1, 500 from t2 on duplicate key update b1 = 400;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t1;`).Check(testkit.Rows("1 400")) @@ -85,97 +86,97 @@ func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk.MustExec(`create table t1(a bigint primary key, b bigint);`) tk.MustExec(`create table t2(a bigint primary key, b bigint);`) _, err := tk.Exec(`insert into t1 select * from t2 on duplicate key update c = t2.b;`) - c.Assert(err.Error(), Equals, `[planner:1054]Unknown column 'c' in 'field list'`) + require.Equal(t, `[planner:1054]Unknown column 'c' in 'field list'`, err.Error()) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(a bigint primary key, b bigint);`) tk.MustExec(`create table t2(a bigint primary key, b bigint);`) _, err = tk.Exec(`insert into t1 select * from t2 on duplicate key update a = b;`) - c.Assert(err.Error(), Equals, `[planner:1052]Column 'b' in field list is ambiguous`) + require.Equal(t, `[planner:1052]Column 'b' in field list is ambiguous`, err.Error()) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(a bigint primary key, b bigint);`) tk.MustExec(`create table t2(a bigint primary key, b bigint);`) _, err = tk.Exec(`insert into t1 select * from t2 on duplicate key update c = b;`) - c.Assert(err.Error(), Equals, `[planner:1054]Unknown column 'c' in 'field list'`) + require.Equal(t, `[planner:1054]Unknown column 'c' in 'field list'`, err.Error()) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(a1 bigint primary key, b1 bigint);`) tk.MustExec(`create table t2(a2 bigint primary key, b2 bigint);`) _, err = tk.Exec(`insert into t1 select * from t2 on duplicate key update a1 = values(b2);`) - c.Assert(err.Error(), Equals, `[planner:1054]Unknown column 'b2' in 'field list'`) + require.Equal(t, `[planner:1054]Unknown column 'b2' in 'field list'`, err.Error()) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(a1 bigint primary key, b1 bigint);`) tk.MustExec(`create table t2(a2 bigint primary key, b2 bigint);`) tk.MustExec(`insert into t1 values(1, 100);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t2 values(1, 200);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t1`).Check(testkit.Rows("1 400")) tk.MustExec(`insert into t1 select * from t2 on duplicate key update b1 = values(b1) + b2;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t1`).Check(testkit.Rows("1 400")) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(k1 bigint, k2 bigint, val bigint, primary key(k1, k2));`) tk.MustExec(`insert into t (val, k1, k2) values (3, 1, 2);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) tk.MustExec(`insert into t (val, k1, k2) select c, a, b from (select 1 as a, 2 as b, 4 as c) tmp on duplicate key update val = tmp.c;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 4`)) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(k1 double, k2 double, v double, primary key(k1, k2));`) tk.MustExec(`insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) tk.MustExec(`insert into t (v, k1, k2) select c, a, b from (select "3" c, "1" a, "2" b) tmp on duplicate key update v=c;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 1 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t;`).Check(testkit.Rows(`1 2 3`)) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(id int, a int, b int);`) tk.MustExec(`insert into t1 values (1, 1, 1);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 values (2, 2, 1);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 values (3, 3, 1);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`create table t2(a int primary key, b int, unique(b));`) tk.MustExec(`insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(5)) + require.Equal(t, uint64(5), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 3 Duplicates: 2 Warnings: 0") tk.MustQuery(`select * from t2 order by a;`).Check(testkit.Rows(`3 1`)) tk.MustExec(`drop table if exists t1, t2;`) tk.MustExec(`create table t1(id int, a int, b int);`) tk.MustExec(`insert into t1 values (1, 1, 1);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 values (2, 1, 2);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`insert into t1 values (3, 3, 1);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.CheckLastMessage("") tk.MustExec(`create table t2(a int primary key, b int, unique(b));`) tk.MustExec(`insert into t2 select a, b from t1 order by id on duplicate key update a=t1.a, b=t1.b;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(4)) + require.Equal(t, uint64(4), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 3 Duplicates: 1 Warnings: 0") tk.MustQuery(`select * from t2 order by a;`).Check(testkit.Rows(`1 2`, `3 1`)) @@ -187,17 +188,17 @@ func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk.MustExec(`insert into t1 values (4, 4, 2, 2);`) tk.MustExec(`create table t2(a int primary key, b int, c int, unique(b), unique(c));`) tk.MustExec(`insert into t2 select a, b, c from t1 order by id on duplicate key update b=t2.b, c=t2.c;`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 4 Duplicates: 0 Warnings: 0") tk.MustQuery(`select * from t2 order by a;`).Check(testkit.Rows(`1 1 1`, `3 2 2`)) tk.MustExec(`drop table if exists t1`) tk.MustExec(`create table t1(a int primary key, b int);`) tk.MustExec(`insert into t1 values(1,1),(2,2),(3,3),(4,4),(5,5);`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(5)) + require.Equal(t, uint64(5), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 5 Duplicates: 0 Warnings: 0") tk.MustExec(`insert into t1 values(4,14),(5,15),(6,16),(7,17),(8,18) on duplicate key update b=b+10`) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(7)) + require.Equal(t, uint64(7), tk.Session().AffectedRows()) tk.CheckLastMessage("Records: 5 Duplicates: 2 Warnings: 0") tk.MustExec("drop table if exists a, b") @@ -206,7 +207,7 @@ func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk.MustExec("insert into a values(1)") tk.MustExec("insert into b values(1, 2)") tk.MustExec("insert into a select x from b ON DUPLICATE KEY UPDATE a.x=b.y") - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.MustQuery("select * from a").Check(testkit.Rows("2")) // Test issue 28078. @@ -217,20 +218,20 @@ func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk.MustExec("insert into a values (1, '2022-01-04 07:02:04', 'a', 1.1), (2, '2022-01-04 07:02:05', 'b', 2.2)") tk.MustExec("insert into b values (2, '12:34:56', 'c', 10), (3, '01:23:45', 'd', 20)") tk.MustExec("insert into a (id) select id from b on duplicate key update a.a2 = b.b2, a.a3 = 3.3") - c.Assert(tk.Se.AffectedRows(), Equals, uint64(3)) + require.Equal(t, uint64(3), tk.Session().AffectedRows()) tk.MustQuery("select * from a").Check(testutil.RowsWithSep("/", "1/2022-01-04 07:02:04/a/1.1", "2/2022-01-04 07:02:05/c/3.3", "3///")) tk.MustExec("insert into a (id) select 4 from b where b3 = 20 on duplicate key update a.a3 = b.b3") - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) tk.MustQuery("select * from a").Check(testutil.RowsWithSep("/", "1/2022-01-04 07:02:04/a/1.1", "2/2022-01-04 07:02:05/c/3.3", "3///", "4///")) tk.MustExec("insert into a (a2, a3) select 'x', 1.2 from b on duplicate key update a.a2 = b.b3") - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) + require.Equal(t, uint64(2), tk.Session().AffectedRows()) tk.MustQuery("select * from a").Check(testutil.RowsWithSep("/", "1/2022-01-04 07:02:04/a/1.1", "2/2022-01-04 07:02:05/c/3.3", @@ -247,29 +248,33 @@ func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { tk.MustQuery(`select * from t1 use index(primary)`).Check(testkit.Rows(`1.0000`)) } -func (s *testSuite8) TestClusterIndexInsertOnDuplicateKey(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterIndexInsertOnDuplicateKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists cluster_index_duplicate_entry_error;") tk.MustExec("create database cluster_index_duplicate_entry_error;") tk.MustExec("use cluster_index_duplicate_entry_error;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t(a char(20), b int, primary key(a));") tk.MustExec("insert into t values('aa', 1), ('bb', 1);") _, err := tk.Exec("insert into t values('aa', 2);") - c.Assert(err, ErrorMatches, ".*Duplicate entry 'aa' for.*") + require.Regexp(t, ".*Duplicate entry 'aa' for.*", err.Error()) tk.MustExec("drop table t;") tk.MustExec("create table t(a char(20), b varchar(30), c varchar(10), primary key(a, b, c));") tk.MustExec("insert into t values ('a', 'b', 'c'), ('b', 'a', 'c');") _, err = tk.Exec("insert into t values ('a', 'b', 'c');") - c.Assert(err, ErrorMatches, ".*Duplicate entry 'a-b-c' for.*") + require.Regexp(t, ".*Duplicate entry 'a-b-c' for.*", err.Error()) } -func (s *testSuite10) TestPaddingCommonHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPaddingCommonHandle(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1;") tk.MustExec(`create table t1(c1 decimal(6,4), primary key(c1))`) tk.MustExec(`insert into t1 set c1 = 0.1`) @@ -277,11 +282,10 @@ func (s *testSuite10) TestPaddingCommonHandle(c *C) { tk.MustQuery(`select * from t1`).Check(testkit.Rows(`1.0000`)) } -func (s *testSuite2) TestInsertReorgDelete(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestInsertReorgDelete(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") inputs := []struct { @@ -332,8 +336,10 @@ func (s *testSuite2) TestInsertReorgDelete(c *C) { } } -func (s *testSuite3) TestUpdateDuplicateKey(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateDuplicateKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) @@ -341,21 +347,23 @@ func (s *testSuite3) TestUpdateDuplicateKey(c *C) { tk.MustExec(`insert into c values(1,2,3);`) tk.MustExec(`insert into c values(1,2,4);`) _, err := tk.Exec(`update c set i=1,j=2,k=4 where i=1 and j=2 and k=3;`) - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-2-4' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1-2-4' for key 'PRIMARY'") } -func (s *testSuite3) TestInsertWrongValueForField(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertWrongValueForField(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t1(a bigint);`) _, err := tk.Exec(`insert into t1 values("asfasdfsajhlkhlksdaf");`) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) + require.True(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField)) tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t1(a varchar(10)) charset ascii;`) _, err = tk.Exec(`insert into t1 values('我');`) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue) + require.True(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField)) tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t1(a char(10) charset utf8);`) @@ -377,12 +385,14 @@ func (s *testSuite3) TestInsertWrongValueForField(c *C) { tk.MustExec(`SET @@sql_mode='STRICT_TRANS_TABLES'`) _, err = tk.Exec(`INSERT INTO ts (id, time1) VALUES (2, TIMESTAMP '1018-12-24 00:00:00')`) - c.Assert(err.Error(), Equals, `[table:1292]Incorrect timestamp value: '1018-12-24 00:00:00' for column 'time1' at row 1`) + require.EqualError(t, err, `[table:1292]Incorrect timestamp value: '1018-12-24 00:00:00' for column 'time1' at row 1`) tk.MustExec(`DROP TABLE ts`) } -func (s *testSuite3) TestInsertValueForCastDecimalField(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertValueForCastDecimalField(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t1(a decimal(15,2));`) @@ -391,8 +401,10 @@ func (s *testSuite3) TestInsertValueForCastDecimalField(c *C) { tk.MustQuery(`select cast(a as decimal) from t1;`).Check(testkit.Rows(`9999999999`)) } -func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertDateTimeWithTimeZone(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test;`) tk.MustExec(`set time_zone="+09:00";`) @@ -493,8 +505,10 @@ func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { } } -func (s *testSuite3) TestInsertZeroYear(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertZeroYear(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t1(a year(4));`) @@ -523,8 +537,10 @@ func (s *testSuite3) TestInsertZeroYear(c *C) { )) } -func (s *testSuiteP1) TestAllowInvalidDates(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAllowInvalidDates(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t1, t2, t3, t4;`) tk.MustExec(`create table t1(d date);`) @@ -558,8 +574,10 @@ func (s *testSuiteP1) TestAllowInvalidDates(c *C) { runWithMode("ALLOW_INVALID_DATES") } -func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertWithAutoidSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t1(id int primary key auto_increment, n int);`) tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int);`) @@ -971,9 +989,9 @@ func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { retryInfo := &variable.RetryInfo{Retrying: true} retryInfo.AddAutoIncrementID(1000) retryInfo.AddAutoIncrementID(1001) - tk.Se.GetSessionVars().RetryInfo = retryInfo + tk.Session().GetSessionVars().RetryInfo = retryInfo tk.MustExec(tt.insert[8:]) - tk.Se.GetSessionVars().RetryInfo = &variable.RetryInfo{} + tk.Session().GetSessionVars().RetryInfo = &variable.RetryInfo{} } else { tk.MustExec(tt.insert) } @@ -987,8 +1005,10 @@ func (s *testSuite3) TestInsertWithAutoidSchema(c *C) { } -func (s *testSuite3) TestPartitionInsertOnDuplicate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionInsertOnDuplicate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t1 (a int,b int,primary key(a,b)) partition by range(a) (partition p0 values less than (100),partition p1 values less than (1000))`) tk.MustExec(`insert into t1 set a=1, b=1`) @@ -1013,33 +1033,38 @@ func (s *testSuite3) TestPartitionInsertOnDuplicate(c *C) { tk.MustQuery("select * from t3").Check(testkit.Rows("1 2 3 4 16")) } -func (s *testSuite3) TestBit(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t1 (a bit(3))`) _, err := tk.Exec("insert into t1 values(-1)") - c.Assert(types.ErrDataTooLong.Equal(err), IsTrue) - c.Assert(err.Error(), Matches, ".*Data too long for column 'a' at.*") + require.True(t, types.ErrDataTooLong.Equal(err)) + require.Regexp(t, ".*Data too long for column 'a' at.*", err.Error()) _, err = tk.Exec("insert into t1 values(9)") - c.Assert(err.Error(), Matches, ".*Data too long for column 'a' at.*") + require.Regexp(t, ".*Data too long for column 'a' at.*", err.Error()) tk.MustExec(`create table t64 (a bit(64))`) tk.MustExec("insert into t64 values(-1)") tk.MustExec("insert into t64 values(18446744073709551615)") // 2^64 - 1 _, err = tk.Exec("insert into t64 values(18446744073709551616)") // z^64 - c.Assert(err.Error(), Matches, ".*Out of range value for column 'a' at.*") + require.Regexp(t, ".*Out of range value for column 'a' at.*", err.Error()) } -func (s *testSuiteP1) TestAllocateContinuousRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAllocateContinuousRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t1 (a int,b int, key I_a(a));`) var wg util.WaitGroupWrapper for i := 0; i < 5; i++ { idx := i wg.Run(func() { - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") for j := 0; j < 10; j++ { k := strconv.Itoa(idx*100 + j) sql := "insert into t1(a,b) values (" + k + ", 2)" @@ -1049,14 +1074,14 @@ func (s *testSuiteP1) TestAllocateContinuousRowID(c *C) { tk.MustExec(sql) q := "select _tidb_rowid from t1 where a=" + k rows := tk.MustQuery(q).Rows() - c.Assert(len(rows), Equals, 21) + require.Equal(t, 21, len(rows)) last := 0 for _, r := range rows { - c.Assert(len(r), Equals, 1) + require.Equal(t, 1, len(r)) v, err := strconv.Atoi(r[0].(string)) - c.Assert(err, Equals, nil) + require.Equal(t, nil, err) if last > 0 { - c.Assert(last+1, Equals, v) + require.Equal(t, v, last+1) } last = v } @@ -1066,48 +1091,54 @@ func (s *testSuiteP1) TestAllocateContinuousRowID(c *C) { wg.Wait() } -func (s *testSuite3) TestJiraIssue5366(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJiraIssue5366(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table bug (a varchar(100))`) tk.MustExec(` insert into bug select ifnull(JSON_UNQUOTE(JSON_EXTRACT('[{"amount":2000,"feeAmount":0,"merchantNo":"20190430140319679394","shareBizCode":"20160311162_SECOND"}]', '$[0].merchantNo')),'') merchant_no union SELECT '20180531557' merchant_no;`) tk.MustQuery(`select * from bug`).Sort().Check(testkit.Rows("20180531557", "20190430140319679394")) } -func (s *testSuite3) TestDMLCast(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDMLCast(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t (a int, b double)`) tk.MustExec(`insert into t values (ifnull('',0)+0, 0)`) tk.MustExec(`insert into t values (0, ifnull('',0)+0)`) tk.MustQuery(`select * from t`).Check(testkit.Rows("0 0", "0 0")) _, err := tk.Exec(`insert into t values ('', 0)`) - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec(`insert into t values (0, '')`) - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec(`update t set a = ''`) - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec(`update t set b = ''`) - c.Assert(err, NotNil) + require.Error(t, err) tk.MustExec("update t set a = ifnull('',0)+0") tk.MustExec("update t set b = ifnull('',0)+0") tk.MustExec("delete from t where a = ''") tk.MustQuery(`select * from t`).Check(testkit.Rows()) } -func (s *testSuite3) TestInsertFloatOverflow(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertFloatOverflow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t,t1;`) tk.MustExec("create table t(col1 FLOAT, col2 FLOAT(10,2), col3 DOUBLE, col4 DOUBLE(10,2), col5 DECIMAL, col6 DECIMAL(10,2));") _, err := tk.Exec("insert into t values (-3.402823466E+68, -34028234.6611, -1.7976931348623157E+308, -17976921.34, -9999999999, -99999999.99);") - c.Assert(err.Error(), Equals, "[types:1264]Out of range value for column 'col1' at row 1") + require.EqualError(t, err, "[types:1264]Out of range value for column 'col1' at row 1") _, err = tk.Exec("insert into t values (-34028234.6611, -3.402823466E+68, -1.7976931348623157E+308, -17976921.34, -9999999999, -99999999.99);") - c.Assert(err.Error(), Equals, "[types:1264]Out of range value for column 'col2' at row 1") + require.EqualError(t, err, "[types:1264]Out of range value for column 'col2' at row 1") _, err = tk.Exec("create table t1(id1 float,id2 float)") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = tk.Exec("insert ignore into t1 values(999999999999999999999999999999999999999,-999999999999999999999999999999999999999)") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery("select @@warning_count").Check(testutil.RowsWithSep("|", "2")) tk.MustQuery("select convert(id1,decimal(65)),convert(id2,decimal(65)) from t1").Check(testkit.Rows("340282346638528860000000000000000000000 -340282346638528860000000000000000000000")) tk.MustExec("drop table if exists t,t1") @@ -1117,12 +1148,14 @@ func (s *testSuite3) TestInsertFloatOverflow(c *C) { // than that of auto_increment_increment, the value of auto_increment_offset is ignored // (https://dev.mysql.com/doc/refman/8.0/en/replication-options-master.html#sysvar_auto_increment_increment), // This issue is a flaw of the implementation of MySQL and it doesn't exist in TiDB. -func (s *testSuite3) TestAutoIDIncrementAndOffset(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoIDIncrementAndOffset(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) // Test for offset is larger than increment. - tk.Se.GetSessionVars().AutoIncrementIncrement = 5 - tk.Se.GetSessionVars().AutoIncrementOffset = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 + tk.Session().GetSessionVars().AutoIncrementOffset = 10 tk.MustExec(`create table io (a int key auto_increment)`) tk.MustExec(`insert into io values (null),(null),(null)`) tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "15", "20")) @@ -1130,31 +1163,31 @@ func (s *testSuite3) TestAutoIDIncrementAndOffset(c *C) { // Test handle is PK. tk.MustExec(`create table io (a int key auto_increment)`) - tk.Se.GetSessionVars().AutoIncrementOffset = 10 - tk.Se.GetSessionVars().AutoIncrementIncrement = 2 + tk.Session().GetSessionVars().AutoIncrementOffset = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 2 tk.MustExec(`insert into io values (),(),()`) tk.MustQuery(`select * from io`).Check(testkit.Rows("10", "12", "14")) tk.MustExec(`delete from io`) // Test reset the increment. - tk.Se.GetSessionVars().AutoIncrementIncrement = 5 + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 tk.MustExec(`insert into io values (),(),()`) tk.MustQuery(`select * from io`).Check(testkit.Rows("15", "20", "25")) tk.MustExec(`delete from io`) - tk.Se.GetSessionVars().AutoIncrementIncrement = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 10 tk.MustExec(`insert into io values (),(),()`) tk.MustQuery(`select * from io`).Check(testkit.Rows("30", "40", "50")) tk.MustExec(`delete from io`) - tk.Se.GetSessionVars().AutoIncrementIncrement = 5 + tk.Session().GetSessionVars().AutoIncrementIncrement = 5 tk.MustExec(`insert into io values (),(),()`) tk.MustQuery(`select * from io`).Check(testkit.Rows("55", "60", "65")) tk.MustExec(`drop table io`) // Test handle is not PK. - tk.Se.GetSessionVars().AutoIncrementIncrement = 2 - tk.Se.GetSessionVars().AutoIncrementOffset = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 2 + tk.Session().GetSessionVars().AutoIncrementOffset = 10 tk.MustExec(`create table io (a int, b int auto_increment, key(b))`) tk.MustExec(`insert into io(b) values (null),(null),(null)`) // AutoID allocation will take increment and offset into consideration. @@ -1163,62 +1196,58 @@ func (s *testSuite3) TestAutoIDIncrementAndOffset(c *C) { tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) tk.MustExec(`delete from io`) - tk.Se.GetSessionVars().AutoIncrementIncrement = 10 + tk.Session().GetSessionVars().AutoIncrementIncrement = 10 tk.MustExec(`insert into io(b) values (null),(null),(null)`) tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) // Test invalid value. - tk.Se.GetSessionVars().AutoIncrementIncrement = -1 - tk.Se.GetSessionVars().AutoIncrementOffset = -2 + tk.Session().GetSessionVars().AutoIncrementIncrement = -1 + tk.Session().GetSessionVars().AutoIncrementOffset = -2 _, err := tk.Exec(`insert into io(b) values (null),(null),(null)`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: -1, auto_increment_offset: -2, both of them must be in range [1..65535]") + require.Error(t, err) + require.EqualError(t, err, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: -1, auto_increment_offset: -2, both of them must be in range [1..65535]") tk.MustExec(`delete from io`) - tk.Se.GetSessionVars().AutoIncrementIncrement = 65536 - tk.Se.GetSessionVars().AutoIncrementOffset = 65536 + tk.Session().GetSessionVars().AutoIncrementIncrement = 65536 + tk.Session().GetSessionVars().AutoIncrementOffset = 65536 _, err = tk.Exec(`insert into io(b) values (null),(null),(null)`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: 65536, auto_increment_offset: 65536, both of them must be in range [1..65535]") -} - -var _ = Suite(&testSuite9{&baseTestSuite{}}) - -type testSuite9 struct { - *baseTestSuite + require.Error(t, err) + require.EqualError(t, err, "[autoid:8060]Invalid auto_increment settings: auto_increment_increment: 65536, auto_increment_offset: 65536, both of them must be in range [1..65535]") } -func (s *testSuite9) TestAutoRandomID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists ar`) tk.MustExec(`create table ar (id bigint key clustered auto_random, name char(10))`) tk.MustExec(`insert into ar(id) values (null)`) rs := tk.MustQuery(`select id from ar`) - c.Assert(len(rs.Rows()), Equals, 1) + require.Equal(t, 1, len(rs.Rows())) firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) + require.NoError(t, err) + require.Greater(t, firstValue, 0) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`delete from ar`) tk.MustExec(`insert into ar(id) values (0)`) rs = tk.MustQuery(`select id from ar`) - c.Assert(len(rs.Rows()), Equals, 1) + require.Equal(t, 1, len(rs.Rows())) firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) + require.NoError(t, err) + require.Greater(t, firstValue, 0) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`delete from ar`) tk.MustExec(`insert into ar(name) values ('a')`) rs = tk.MustQuery(`select id from ar`) - c.Assert(len(rs.Rows()), Equals, 1) + require.Equal(t, 1, len(rs.Rows())) firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) + require.NoError(t, err) + require.Greater(t, firstValue, 0) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`drop table ar`) @@ -1226,53 +1255,57 @@ func (s *testSuite9) TestAutoRandomID(c *C) { overflowVal := 1 << (64 - 5) errMsg := fmt.Sprintf(autoid.AutoRandomRebaseOverflow, overflowVal, 1<<(64-16)-1) _, err = tk.Exec(fmt.Sprintf("alter table ar auto_random_base = %d", overflowVal)) - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), errMsg), IsTrue) + require.Error(t, err) + require.True(t, strings.Contains(err.Error(), errMsg)) } -func (s *testSuite9) TestMultiAutoRandomID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiAutoRandomID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists ar`) tk.MustExec(`create table ar (id bigint key clustered auto_random, name char(10))`) tk.MustExec(`insert into ar(id) values (null),(null),(null)`) rs := tk.MustQuery(`select id from ar order by id`) - c.Assert(len(rs.Rows()), Equals, 3) + require.Equal(t, 3, len(rs.Rows())) firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) - c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1)) - c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2)) + require.NoError(t, err) + require.Greater(t, firstValue, 0) + require.Equal(t, fmt.Sprintf("%d", firstValue+1), rs.Rows()[1][0].(string)) + require.Equal(t, fmt.Sprintf("%d", firstValue+2), rs.Rows()[2][0].(string)) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`delete from ar`) tk.MustExec(`insert into ar(id) values (0),(0),(0)`) rs = tk.MustQuery(`select id from ar order by id`) - c.Assert(len(rs.Rows()), Equals, 3) + require.Equal(t, 3, len(rs.Rows())) firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) - c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1)) - c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2)) + require.NoError(t, err) + require.Greater(t, firstValue, 0) + require.Equal(t, fmt.Sprintf("%d", firstValue+1), rs.Rows()[1][0].(string)) + require.Equal(t, fmt.Sprintf("%d", firstValue+2), rs.Rows()[2][0].(string)) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`delete from ar`) tk.MustExec(`insert into ar(name) values ('a'),('a'),('a')`) rs = tk.MustQuery(`select id from ar order by id`) - c.Assert(len(rs.Rows()), Equals, 3) + require.Equal(t, 3, len(rs.Rows())) firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) - c.Assert(rs.Rows()[1][0].(string), Equals, fmt.Sprintf("%d", firstValue+1)) - c.Assert(rs.Rows()[2][0].(string), Equals, fmt.Sprintf("%d", firstValue+2)) + require.NoError(t, err) + require.Greater(t, firstValue, 0) + require.Equal(t, fmt.Sprintf("%d", firstValue+1), rs.Rows()[1][0].(string)) + require.Equal(t, fmt.Sprintf("%d", firstValue+2), rs.Rows()[2][0].(string)) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`drop table ar`) } -func (s *testSuite9) TestAutoRandomIDAllowZero(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomIDAllowZero(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists ar`) tk.MustExec(`create table ar (id bigint key clustered auto_random, name char(10))`) @@ -1283,26 +1316,28 @@ func (s *testSuite9) TestAutoRandomIDAllowZero(c *C) { tk.MustExec(`insert into ar(id) values (0)`) rs = tk.MustQuery(`select id from ar`) - c.Assert(len(rs.Rows()), Equals, 1) + require.Equal(t, 1, len(rs.Rows())) firstValue, err := strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, firstValue) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`delete from ar`) tk.MustExec(`insert into ar(id) values (null)`) rs = tk.MustQuery(`select id from ar`) - c.Assert(len(rs.Rows()), Equals, 1) + require.Equal(t, 1, len(rs.Rows())) firstValue, err = strconv.Atoi(rs.Rows()[0][0].(string)) - c.Assert(err, IsNil) - c.Assert(firstValue, Greater, 0) + require.NoError(t, err) + require.Greater(t, firstValue, 0) tk.MustQuery(`select last_insert_id()`).Check(testkit.Rows(fmt.Sprintf("%d", firstValue))) tk.MustExec(`drop table ar`) } -func (s *testSuite9) TestAutoRandomIDExplicit(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomIDExplicit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@allow_auto_random_explicit_insert = true") tk.MustExec(`use test`) @@ -1322,37 +1357,37 @@ func (s *testSuite9) TestAutoRandomIDExplicit(c *C) { tk.MustExec(`drop table ar`) } -func (s *testSuite9) TestInsertErrorMsg(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertErrorMsg(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t`) tk.MustExec(`create table t (a int primary key, b datetime, d date)`) _, err := tk.Exec(`insert into t values (1, '2019-02-11 30:00:00', '2019-01-31')`) - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "Incorrect datetime value: '2019-02-11 30:00:00' for column 'b' at row 1"), IsTrue, Commentf("%v", err)) + require.Error(t, err) + require.Contains(t, err.Error(), "Incorrect datetime value: '2019-02-11 30:00:00' for column 'b' at row 1") } -func (s *testSuite9) TestIssue16366(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16366(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test;`) tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(c numeric primary key);`) tk.MustExec("insert ignore into t values(null);") _, err := tk.Exec(`insert into t values(0);`) - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "Duplicate entry '0' for key 'PRIMARY'"), IsTrue, Commentf("%v", err)) + require.Error(t, err) + require.Contains(t, err.Error(), "Duplicate entry '0' for key 'PRIMARY'") } -var _ = SerialSuites(&testSuite10{&baseTestSuite{}}) - -type testSuite10 struct { - *baseTestSuite -} - -func (s *testSuite10) TestClusterPrimaryTablePlainInsert(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterPrimaryTablePlainInsert(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists t1pk`) tk.MustExec(`create table t1pk(id varchar(200) primary key, v int)`) @@ -1391,10 +1426,12 @@ func (s *testSuite10) TestClusterPrimaryTablePlainInsert(c *C) { Check(testkit.Rows("abc xyz 1 100", "abc xyz 1 101", "abc zzz 1 101")) } -func (s *testSuite10) TestClusterPrimaryTableInsertIgnore(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterPrimaryTableInsertIgnore(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists it1pk`) tk.MustExec(`create table it1pk(id varchar(200) primary key, v int)`) @@ -1417,10 +1454,12 @@ func (s *testSuite10) TestClusterPrimaryTableInsertIgnore(c *C) { tk.MustQuery(`select * from it1pku`).Check(testkit.Rows("abc 1 2", "bbb 2 1")) } -func (s *testSuite10) TestClusterPrimaryTableInsertDuplicate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterPrimaryTableInsertDuplicate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists dt1pi`) tk.MustExec(`create table dt1pi(id varchar(200) primary key, v int)`) @@ -1449,10 +1488,12 @@ func (s *testSuite10) TestClusterPrimaryTableInsertDuplicate(c *C) { tk.MustQuery(`select id1, id2, v from ts1pk`).Check(testkit.Rows("2018-01-01 11:11:12 2018-01-01 11:11:11 2")) } -func (s *testSuite10) TestClusterPrimaryKeyForIndexScan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterPrimaryKeyForIndexScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists pkt1;") tk.MustExec("CREATE TABLE pkt1 (a varchar(255), b int, index idx(b), primary key(a,b));") @@ -1478,10 +1519,10 @@ func (s *testSuite10) TestClusterPrimaryKeyForIndexScan(c *C) { tk.MustExec(sql) cnt++ } - c.Assert(cnt, Equals, 15) + require.Equal(t, 15, cnt) } -func (s *testSuite10) TestInsertRuntimeStat(c *C) { +func TestInsertRuntimeStat(t *testing.T) { stats := &executor.InsertRuntimeStat{ BasicRuntimeStats: &execdetails.BasicRuntimeStats{}, SnapshotRuntimeStats: nil, @@ -1489,20 +1530,19 @@ func (s *testSuite10) TestInsertRuntimeStat(c *C) { Prefetch: 1 * time.Second, } stats.BasicRuntimeStats.Record(5*time.Second, 1) - c.Assert(stats.String(), Equals, "prepare: 3s, check_insert: {total_time: 2s, mem_insert_time: 1s, prefetch: 1s}") - c.Assert(stats.String(), Equals, stats.Clone().String()) + require.Equal(t, "prepare: 3s, check_insert: {total_time: 2s, mem_insert_time: 1s, prefetch: 1s}", stats.String()) + require.Equal(t, stats.Clone().String(), stats.String()) stats.Merge(stats.Clone()) - c.Assert(stats.String(), Equals, "prepare: 6s, check_insert: {total_time: 4s, mem_insert_time: 2s, prefetch: 2s}") + require.Equal(t, "prepare: 6s, check_insert: {total_time: 4s, mem_insert_time: 2s, prefetch: 2s}", stats.String()) } -func (s *testSerialSuite) TestDuplicateEntryMessage(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) +func TestDuplicateEntryMessage(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") for _, enable := range []variable.ClusteredIndexDefMode{variable.ClusteredIndexDefModeOn, variable.ClusteredIndexDefModeOff, variable.ClusteredIndexDefModeIntOnly} { - tk.Se.GetSessionVars().EnableClusteredIndex = enable + tk.Session().GetSessionVars().EnableClusteredIndex = enable tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b char(10), unique key(b)) collate utf8mb4_general_ci;") tk.MustExec("insert into t value (34, '12Ak');") @@ -1563,8 +1603,10 @@ func (s *testSerialSuite) TestDuplicateEntryMessage(c *C) { } } -func (s *testSerialSuite) TestIssue20768(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20768(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a year, primary key(a))") @@ -1579,18 +1621,21 @@ func (s *testSerialSuite) TestIssue20768(c *C) { tk.MustQuery("select /*+ merge_join(t1) */ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows("0 0")) } -func (s *testSuite9) TestIssue10402(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue10402(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table vctt (v varchar(4), c char(4))") tk.MustExec("insert into vctt values ('ab ', 'ab ')") tk.MustQuery("select * from vctt").Check(testkit.Rows("ab ab")) tk.MustExec("delete from vctt") - tk.Se.GetSessionVars().StmtCtx.SetWarnings(nil) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) tk.MustExec("insert into vctt values ('ab\\n\\n\\n', 'ab\\n\\n\\n'), ('ab\\t\\t\\t', 'ab\\t\\t\\t'), ('ab ', 'ab '), ('ab\\r\\r\\r', 'ab\\r\\r\\r')") - c.Check(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(4)) - warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Check(fmt.Sprintf("%v", warns), Equals, "[{Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 6} {Warning [types:1265]Data truncated, field len 4, data len 5}]") + require.Equal(t, uint16(4), tk.Session().GetSessionVars().StmtCtx.WarningCount()) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Equal(t, "[{Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 6} {Warning [types:1265]Data truncated, field len 4, data len 5}]", + fmt.Sprintf("%v", warns)) tk.MustQuery("select * from vctt").Check(testkit.Rows("ab\n\n ab\n\n", "ab\t\t ab\t\t", "ab ab", "ab\r\r ab\r\r")) tk.MustQuery("select length(v), length(c) from vctt").Check(testkit.Rows("4 4", "4 4", "4 2", "4 4")) } @@ -1614,23 +1659,27 @@ func combination(items []string) func() []string { } // TestDuplicatedEntryErr See https://github.com/pingcap/tidb/issues/24582 -func (s *testSuite10) TestDuplicatedEntryErr(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDuplicatedEntryErr(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(a int, b varchar(20), primary key(a,b(3)) clustered);") tk.MustExec("insert into t1 values(1,'aaaaa');") err := tk.ExecToErr("insert into t1 values(1,'aaaaa');") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-aaa' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1-aaa' for key 'PRIMARY'") err = tk.ExecToErr("insert into t1 select 1, 'aaa'") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-aaa' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1-aaa' for key 'PRIMARY'") tk.MustExec("insert into t1 select 1, 'bb'") err = tk.ExecToErr("insert into t1 select 1, 'bb'") - c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-bb' for key 'PRIMARY'") + require.EqualError(t, err, "[kv:1062]Duplicate entry '1-bb' for key 'PRIMARY'") } -func (s *testSuite10) TestBinaryLiteralInsertToEnum(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBinaryLiteralInsertToEnum(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists bintest") @@ -1639,8 +1688,10 @@ func (s *testSuite10) TestBinaryLiteralInsertToEnum(c *C) { tk.MustQuery("select * from bintest").Check(testkit.Rows("a")) } -func (s *testSuite10) TestBinaryLiteralInsertToSet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBinaryLiteralInsertToSet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists bintest") @@ -1649,14 +1700,10 @@ func (s *testSuite10) TestBinaryLiteralInsertToSet(c *C) { tk.MustQuery("select * from bintest").Check(testkit.Rows("a")) } -var _ = SerialSuites(&testSuite13{&baseTestSuite{}}) - -type testSuite13 struct { - *baseTestSuite -} - -func (s *testSuite13) TestGlobalTempTableAutoInc(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGlobalTempTableAutoInc(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") @@ -1700,8 +1747,10 @@ func (s *testSuite13) TestGlobalTempTableAutoInc(c *C) { tk.MustExec("commit") } -func (s *testSuite13) TestGlobalTempTableRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGlobalTempTableRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") tk.MustExec("create global temporary table temp_test(id int) on commit delete rows") @@ -1735,8 +1784,10 @@ func (s *testSuite13) TestGlobalTempTableRowID(c *C) { tk.MustExec("commit") } -func (s *testSuite13) TestGlobalTempTableParallel(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGlobalTempTableParallel(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") @@ -1747,7 +1798,8 @@ func (s *testSuite13) TestGlobalTempTableParallel(c *C) { var wg util.WaitGroupWrapper insertFunc := func() { - newTk := testkit.NewTestKitWithInit(c, s.store) + newTk := testkit.NewTestKit(t, store) + newTk.MustExec("use test") newTk.MustExec("begin") for i := 0; i < loops; i++ { newTk.MustExec("insert temp_test value(0)") @@ -1764,13 +1816,15 @@ func (s *testSuite13) TestGlobalTempTableParallel(c *C) { wg.Wait() } -func (s *testSuite13) TestIssue26762(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26762(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 date);") _, err := tk.Exec("insert into t1 values('2020-02-31');") - c.Assert(err.Error(), Equals, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`) + require.EqualError(t, err, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`) tk.MustExec("set @@sql_mode='ALLOW_INVALID_DATES';") tk.MustExec("insert into t1 values('2020-02-31');") @@ -1778,11 +1832,13 @@ func (s *testSuite13) TestIssue26762(c *C) { tk.MustExec("set @@sql_mode='STRICT_TRANS_TABLES';") _, err = tk.Exec("insert into t1 values('2020-02-31');") - c.Assert(err.Error(), Equals, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`) + require.EqualError(t, err, `[table:1292]Incorrect date value: '2020-02-31' for column 'c1' at row 1`) } -func (s *testSuite10) TestStringtoDecimal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStringtoDecimal(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id decimal(10))") @@ -1801,8 +1857,10 @@ func (s *testSuite10) TestStringtoDecimal(c *C) { tk.MustExec("drop table if exists t") } -func (s *testSuite13) TestIssue17745(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue17745(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("drop table if exists tt1") tk.MustExec("create table tt1 (c1 decimal(64))") @@ -1819,8 +1877,10 @@ func (s *testSuite13) TestIssue17745(c *C) { } // TestInsertIssue29892 test the double type with auto_increment problem, just leverage the serial test suite. -func (s *testAutoRandomSuite) TestInsertIssue29892(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInsertIssue29892(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec("set global tidb_txn_mode='optimistic';") @@ -1829,12 +1889,12 @@ func (s *testAutoRandomSuite) TestInsertIssue29892(c *C) { tk.MustExec("create table t(a double auto_increment key, b int)") tk.MustExec("insert into t values (146576794, 1)") - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec(`use test`) tk1.MustExec("begin") tk1.MustExec("insert into t(b) select 1") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec(`use test`) tk2.MustExec("begin") tk2.MustExec("insert into t values (146576795, 1)") @@ -1844,13 +1904,15 @@ func (s *testAutoRandomSuite) TestInsertIssue29892(c *C) { // since the origin auto-id (146576795) is cached in retryInfo, it will be fetched again to do the retry again, // which will duplicate with what has been inserted in tk1. _, err := tk1.Exec("commit") - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "Duplicate entry"), Equals, true) + require.Error(t, err) + require.Equal(t, true, strings.Contains(err.Error(), "Duplicate entry")) } // https://github.com/pingcap/tidb/issues/29483. -func (s *testSuite13) TestReplaceAllocatingAutoID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReplaceAllocatingAutoID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists replace_auto_id;") tk.MustExec("create database replace_auto_id;") tk.MustExec(`use replace_auto_id;`) diff --git a/executor/inspection_profile.go b/executor/inspection_profile.go index 605f2d0efad37..15885010dce25 100644 --- a/executor/inspection_profile.go +++ b/executor/inspection_profile.go @@ -167,12 +167,7 @@ func (n *metricNode) getLabelValue(label string) *metricValue { func (n *metricNode) queryRowsByLabel(pb *profileBuilder, query string, handleRowFn func(label string, v float64)) error { exec := pb.sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, query) - if err != nil { - return err - } - - rows, _, err := pb.sctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, query) if err != nil { return err } diff --git a/executor/inspection_result.go b/executor/inspection_result.go index a8e06d7c55b3f..debcf723a3e64 100644 --- a/executor/inspection_result.go +++ b/executor/inspection_result.go @@ -140,10 +140,7 @@ func (e *inspectionResultRetriever) retrieve(ctx context.Context, sctx sessionct e.statusToInstanceAddress = make(map[string]string) var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "select instance,status_address from information_schema.cluster_info;") - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select instance,status_address from information_schema.cluster_info;") if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("get cluster info failed: %v", err)) } @@ -249,22 +246,14 @@ func (configInspection) inspectDiffConfig(ctx context.Context, sctx sessionctx.C "storage.data-dir", "storage.block-cache.capacity", } - var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "select type, `key`, count(distinct value) as c from information_schema.cluster_config where `key` not in (%?) group by type, `key` having c > 1", ignoreConfigKey) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select type, `key`, count(distinct value) as c from information_schema.cluster_config where `key` not in (%?) group by type, `key` having c > 1", ignoreConfigKey) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration consistency failed: %v", err)) } generateDetail := func(tp, item string) string { - var rows []chunk.Row - stmt, err := exec.ParseWithParams(ctx, true, "select value, instance from information_schema.cluster_config where type=%? and `key`=%?;", tp, item) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select value, instance from information_schema.cluster_config where type=%? and `key`=%?;", tp, item) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration consistency failed: %v", err)) return fmt.Sprintf("the cluster has different config value of %[2]s, execute the sql to see more detail: select * from information_schema.cluster_config where type='%[1]s' and `key`='%[2]s'", @@ -347,7 +336,7 @@ func (c configInspection) inspectCheckConfig(ctx context.Context, sctx sessionct } sql.Reset() fmt.Fprintf(sql, "select type,instance,value from information_schema.%s where %s", cas.table, cas.cond) - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) + stmt, err := exec.ParseWithParams(ctx, sql.String()) if err == nil { rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) } @@ -376,12 +365,8 @@ func (c configInspection) checkTiKVBlockCacheSizeConfig(ctx context.Context, sct if !filter.enable(item) { return nil } - var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "select instance,value from information_schema.cluster_config where type='tikv' and `key` = 'storage.block-cache.capacity'") - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select instance,value from information_schema.cluster_config where type='tikv' and `key` = 'storage.block-cache.capacity'") if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration in reason failed: %v", err)) } @@ -405,10 +390,7 @@ func (c configInspection) checkTiKVBlockCacheSizeConfig(ctx context.Context, sct ipToCount[ip]++ } - stmt, err = exec.ParseWithParams(ctx, true, "select instance, value from metrics_schema.node_total_memory where time=now()") - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err = exec.ExecRestrictedSQL(ctx, nil, "select instance, value from metrics_schema.node_total_memory where time=now()") if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check configuration in reason failed: %v", err)) } @@ -471,12 +453,8 @@ func (configInspection) convertReadableSizeToByteSize(sizeStr string) (uint64, e func (versionInspection) inspect(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter) []inspectionResult { exec := sctx.(sqlexec.RestrictedSQLExecutor) - var rows []chunk.Row // check the configuration consistent - stmt, err := exec.ParseWithParams(ctx, true, "select type, count(distinct git_hash) as c from information_schema.cluster_info group by type having c > 1;") - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "select type, count(distinct git_hash) as c from information_schema.cluster_info group by type having c > 1;") if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("check version consistency failed: %v", err)) } @@ -630,7 +608,6 @@ func (criticalErrorInspection) inspectError(ctx context.Context, sctx sessionctx condition := filter.timeRange.Condition() var results []inspectionResult - var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) sql := new(strings.Builder) for _, rule := range rules { @@ -643,10 +620,7 @@ func (criticalErrorInspection) inspectError(ctx context.Context, sctx sessionctx sql.Reset() fmt.Fprintf(sql, "select `%[1]s`,sum(value) as total from `%[2]s`.`%[3]s` %[4]s group by `%[1]s` having total>=1.0", strings.Join(def.Labels, "`,`"), util.MetricSchemaName.L, rule.tbl, condition) - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) continue @@ -697,11 +671,7 @@ func (criticalErrorInspection) inspectForServerDown(ctx context.Context, sctx se fmt.Fprintf(sql, `select t1.job,t1.instance, t2.min_time from (select instance,job from metrics_schema.up %[1]s group by instance,job having max(value)-min(value)>0) as t1 join (select instance,min(time) as min_time from metrics_schema.up %[1]s and value=0 group by instance,job) as t2 on t1.instance=t2.instance order by job`, condition) - var rows []chunk.Row - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) } @@ -726,10 +696,7 @@ func (criticalErrorInspection) inspectForServerDown(ctx context.Context, sctx se // Check from log. sql.Reset() fmt.Fprintf(sql, "select type,instance,time from information_schema.cluster_log %s and level = 'info' and message like '%%Welcome to'", condition) - stmt, err = exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err = exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) } @@ -843,7 +810,6 @@ func (thresholdCheckInspection) inspectThreshold1(ctx context.Context, sctx sess condition := filter.timeRange.Condition() var results []inspectionResult - var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) sql := new(strings.Builder) for _, rule := range rules { @@ -863,10 +829,7 @@ func (thresholdCheckInspection) inspectThreshold1(ctx context.Context, sctx sess (select instance, max(value) as cpu from metrics_schema.tikv_thread_cpu %[3]s and name like '%[1]s' group by instance) as t1 where t1.cpu > %[2]f;`, rule.component, rule.threshold, condition) } - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) continue @@ -1016,7 +979,6 @@ func (thresholdCheckInspection) inspectThreshold2(ctx context.Context, sctx sess condition := filter.timeRange.Condition() var results []inspectionResult - var rows []chunk.Row sql := new(strings.Builder) exec := sctx.(sqlexec.RestrictedSQLExecutor) for _, rule := range rules { @@ -1036,10 +998,7 @@ func (thresholdCheckInspection) inspectThreshold2(ctx context.Context, sctx sess } else { fmt.Fprintf(sql, "select instance, max(value)/%.0f as max_value from metrics_schema.%s %s group by instance having max_value > %f;", rule.factor, rule.tbl, cond, rule.threshold) } - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) continue @@ -1215,17 +1174,13 @@ func (thresholdCheckInspection) inspectThreshold3(ctx context.Context, sctx sess func checkRules(ctx context.Context, sctx sessionctx.Context, filter inspectionFilter, rules []ruleChecker) []inspectionResult { var results []inspectionResult - var rows []chunk.Row exec := sctx.(sqlexec.RestrictedSQLExecutor) for _, rule := range rules { if !filter.enable(rule.getItem()) { continue } sql := rule.genSQL(filter.timeRange) - stmt, err := exec.ParseWithParams(ctx, true, sql) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) continue @@ -1244,11 +1199,7 @@ func (c thresholdCheckInspection) inspectForLeaderDrop(ctx context.Context, sctx fmt.Fprintf(sql, `select address,min(value) as mi,max(value) as mx from metrics_schema.pd_scheduler_store_status %s and type='leader_count' group by address having mx-mi>%v`, condition, threshold) exec := sctx.(sqlexec.RestrictedSQLExecutor) - var rows []chunk.Row - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) return nil @@ -1259,10 +1210,7 @@ func (c thresholdCheckInspection) inspectForLeaderDrop(ctx context.Context, sctx sql.Reset() fmt.Fprintf(sql, `select time, value from metrics_schema.pd_scheduler_store_status %s and type='leader_count' and address = '%s' order by time`, condition, address) var subRows []chunk.Row - stmt, err := exec.ParseWithParams(ctx, true, sql.String()) - if err == nil { - subRows, _, err = exec.ExecRestrictedStmt(ctx, stmt) - } + subRows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql.String()) if err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("execute '%s' failed: %v", sql, err)) continue diff --git a/executor/inspection_summary.go b/executor/inspection_summary.go index 2c58248ed4a03..ebd3f69abc4f8 100644 --- a/executor/inspection_summary.go +++ b/executor/inspection_summary.go @@ -460,11 +460,7 @@ func (e *inspectionSummaryRetriever) retrieve(ctx context.Context, sctx sessionc util.MetricSchemaName.L, name, cond) } exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, sql) - if err != nil { - return nil, errors.Errorf("execute '%s' failed: %v", sql, err) - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql) if err != nil { return nil, errors.Errorf("execute '%s' failed: %v", sql, err) } diff --git a/executor/join_pkg_test.go b/executor/join_pkg_test.go index 75af33d7acb5d..079010706db5f 100644 --- a/executor/join_pkg_test.go +++ b/executor/join_pkg_test.go @@ -25,6 +25,13 @@ import ( "github.com/pingcap/tidb/types" ) +var _ = Suite(&pkgTestSuite{}) +var _ = SerialSuites(&pkgTestSerialSuite{}) + +type pkgTestSuite struct{} + +type pkgTestSerialSuite struct{} + func (s *pkgTestSerialSuite) TestJoinExec(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testRowContainerSpill", "return(true)"), IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testRowContainerSpill"), IsNil) }() diff --git a/executor/join_test.go b/executor/join_test.go index d607a9b3062da..51d8b90e9c431 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -19,44 +19,32 @@ import ( "fmt" "math/rand" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/executor" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -type testSuiteJoin1 struct { - *baseTestSuite -} - -type testSuiteJoin2 struct { - *baseTestSuite -} - -type testSuiteJoin3 struct { - *baseTestSuite -} - -type testSuiteJoinSerial struct { - *baseTestSuite -} - -func (s *testSuiteJoin1) TestJoinPanic(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinPanic2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") tk.MustExec("drop table if exists events") tk.MustExec("create table events (clock int, source int)") tk.MustQuery("SELECT * FROM events e JOIN (SELECT MAX(clock) AS clock FROM events e2 GROUP BY e2.source) e3 ON e3.clock=e.clock") err := tk.ExecToErr("SELECT * FROM events e JOIN (SELECT clock FROM events e2 GROUP BY e2.source) e3 ON e3.clock=e.clock") - c.Check(err, NotNil) + require.Error(t, err) // Test for PR 18983, use to detect race. tk.MustExec("use test") @@ -68,20 +56,24 @@ func (s *testSuiteJoin1) TestJoinPanic(c *C) { tk.MustQuery("select tpj1.b,tpj2.b from tpj1 left join tpj2 on tpj1.id=tpj2.id where tpj1.id=1;").Check(testkit.Rows("1 1")) } -func (s *testSuite) TestJoinInDisk(c *C) { - defer config.RestoreFunc()() +func TestJoinInDisk(t *testing.T) { + origin := config.RestoreFunc() + defer origin() config.UpdateGlobal(func(conf *config.Config) { conf.OOMUseTmpStorage = true + conf.OOMAction = config.OOMActionLog }) - tk := testkit.NewTestKit(c, s.store) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") sm := &mockSessionManager1{ PS: make([]*util.ProcessInfo, 0), } - tk.Se.SetSessionManager(sm) - s.domain.ExpensiveQueryHandle().SetSessionManager(sm) + tk.Session().SetSessionManager(sm) + dom.ExpensiveQueryHandle().SetSessionManager(sm) // TODO(fengliyuan): how to ensure that it is using disk really? tk.MustExec("set @@tidb_mem_quota_query=1;") @@ -95,14 +87,16 @@ func (s *testSuite) TestJoinInDisk(c *C) { result.Check(testkit.Rows("2 2 2 3")) } -func (s *testSuiteJoin2) TestJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoin2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_index_lookup_join_concurrency = 200") - c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency(), Equals, 200) + require.Equal(t, 200, tk.Session().GetSessionVars().IndexLookupJoinConcurrency()) tk.MustExec("set @@tidb_index_lookup_join_concurrency = 4") - c.Assert(tk.Se.GetSessionVars().IndexLookupJoinConcurrency(), Equals, 4) + require.Equal(t, 4, tk.Session().GetSessionVars().IndexLookupJoinConcurrency()) tk.MustExec("set @@tidb_index_lookup_size = 2") tk.MustExec("use test") @@ -218,11 +212,11 @@ func (s *testSuiteJoin2) TestJoin(c *C) { // Test that two conflict hints will return warning. tk.MustExec("select /*+ TIDB_INLJ(t) TIDB_SMJ(t) */ * from t join t1 on t.a=t1.a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) tk.MustExec("select /*+ TIDB_INLJ(t) TIDB_HJ(t) */ * from t join t1 on t.a=t1.a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) tk.MustExec("select /*+ TIDB_SMJ(t) TIDB_HJ(t) */ * from t join t1 on t.a=t1.a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -352,8 +346,10 @@ func (s *testSuiteJoin2) TestJoin(c *C) { tk.MustQuery("select min(t2.b) from t1 right join t2 on t2.a=t1.a right join t3 on t2.a=t3.a left join t4 on t3.a=t4.a").Check(testkit.Rows("1")) } -func (s *testSuiteJoin2) TestJoinCast(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinCast(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) var result *testkit.Result tk.MustExec("use test") @@ -423,7 +419,7 @@ func (s *testSuiteJoin2) TestJoinCast(c *C) { tk.MustExec("insert into t value(18446744073709551615);") tk.MustExec("insert into t1 value(-1);") result = tk.MustQuery("select * from t, t1 where t.c1 = t1.c1;") - c.Check(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) /* Issues 11896 */ tk.MustExec("drop table if exists t;") @@ -433,7 +429,7 @@ func (s *testSuiteJoin2) TestJoinCast(c *C) { tk.MustExec("insert into t value(1);") tk.MustExec("insert into t1 value(1);") result = tk.MustQuery("select * from t, t1 where t.c1 = t1.c1;") - c.Check(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") @@ -443,7 +439,7 @@ func (s *testSuiteJoin2) TestJoinCast(c *C) { tk.MustExec("insert into t1 value(18446744073709551615);") result = tk.MustQuery("select * from t, t1 where t.c1 = t1.c1;") // TODO: MySQL will return one row, because c1 in t1 is 0xffffffff, which equals to -1. - c.Check(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists t1") @@ -538,8 +534,10 @@ func (s *testSuiteJoin2) TestJoinCast(c *C) { tk.MustExec("set @@tidb_init_chunk_size=32") } -func (s *testSuiteJoin1) TestUsing(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUsing(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3, t4") @@ -656,8 +654,10 @@ func (s *testSuiteJoin1) TestUsing(c *C) { tk.MustQuery("select t1.t0, t2.t0 from t1 join t2 using(t0) having t1.t0 > 0").Check(testkit.Rows("1 1")) } -func (s *testSuiteWithData) TestUsingAndNaturalJoinSchema(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUsingAndNaturalJoinSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3, t4") tk.MustExec("create table t1 (c int, b int);") @@ -679,18 +679,20 @@ func (s *testSuiteWithData) TestUsingAndNaturalJoinSchema(c *C) { SQL string Res []string } - s.testData.GetTestCases(c, &input, &output) + executorSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) } } -func (s *testSuiteWithData) TestNaturalJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNaturalJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -705,20 +707,22 @@ func (s *testSuiteWithData) TestNaturalJoin(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + executorSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) } } -func (s *testSuiteJoin3) TestMultiJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t35(a35 int primary key, b35 int, x35 int)") tk.MustExec("create table t40(a40 int primary key, b40 int, x40 int)") @@ -809,8 +813,10 @@ AND b44=a42`) result.Check(testkit.Rows("7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7 7")) } -func (s *testSuiteJoin3) TestSubquerySameTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSubquerySameTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int)") @@ -821,8 +827,10 @@ func (s *testSuiteJoin3) TestSubquerySameTable(c *C) { result.Check(testkit.Rows("1")) } -func (s *testSuiteJoin3) TestSubquery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("set @@tidb_hashagg_partial_concurrency=1") tk.MustExec("set @@tidb_hashagg_final_concurrency=1") @@ -911,10 +919,10 @@ func (s *testSuiteJoin3) TestSubquery(c *C) { result = tk.MustQuery("select (select t.id from t where s.id < 2 and t.id = s.id) from t s") result.Sort().Check(testkit.Rows("1", "", "")) rs, err := tk.Exec("select (select t.id from t where t.id = t.v and t.v != s.id) from t s") - c.Check(err, IsNil) - _, err = session.GetRows4Test(context.Background(), tk.Se, rs) - c.Check(err, NotNil) - c.Check(rs.Close(), IsNil) + require.NoError(t, err) + _, err = session.GetRows4Test(context.Background(), tk.Session(), rs) + require.Error(t, err) + require.NoError(t, rs.Close()) tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists s") @@ -1060,8 +1068,10 @@ func (s *testSuiteJoin3) TestSubquery(c *C) { tk.MustExec("set @@tidb_hash_join_concurrency=5") } -func (s *testSuiteJoin1) TestInSubquery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int)") @@ -1127,8 +1137,10 @@ func (s *testSuiteJoin1) TestInSubquery(c *C) { result.Check(testkit.Rows("2", "2", "1")) } -func (s *testSuiteJoin1) TestJoinLeak(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinLeak(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_hash_join_concurrency=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1139,18 +1151,20 @@ func (s *testSuiteJoin1) TestJoinLeak(c *C) { } tk.MustExec("commit") result, err := tk.Exec("select * from t t1 left join (select 1) t2 on 1") - c.Assert(err, IsNil) + require.NoError(t, err) req := result.NewChunk(nil) err = result.Next(context.Background(), req) - c.Assert(err, IsNil) + require.NoError(t, err) time.Sleep(time.Millisecond) - c.Assert(result.Close(), IsNil) + require.NoError(t, result.Close()) tk.MustExec("set @@tidb_hash_join_concurrency=5") } -func (s *testSuiteJoin1) TestHashJoinExecEncodeDecodeRow(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashJoinExecEncodeDecodeRow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") @@ -1162,8 +1176,10 @@ func (s *testSuiteJoin1) TestHashJoinExecEncodeDecodeRow(c *C) { result.Check(testkit.Rows("2003-06-09 10:51:26")) } -func (s *testSuiteJoin1) TestSubqueryInJoinOn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSubqueryInJoinOn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") @@ -1173,11 +1189,13 @@ func (s *testSuiteJoin1) TestSubqueryInJoinOn(c *C) { tk.MustExec("insert into t2 values (1)") err := tk.ExecToErr("SELECT * FROM t1 JOIN t2 on (t2.id < all (SELECT 1))") - c.Check(err, NotNil) + require.Error(t, err) } -func (s *testSuiteJoin1) TestIssue5255(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue5255(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b date, c float, primary key(a, b))") @@ -1189,8 +1207,10 @@ func (s *testSuiteJoin1) TestIssue5255(c *C) { tk.MustQuery("select /*+ INL_MERGE_JOIN(t1) */ * from t1 join t2 on t1.a=t2.a").Check(testkit.Rows("1 2017-11-29 2.2 1")) } -func (s *testSuiteJoin1) TestIssue5278(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue5278(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, tt") tk.MustExec("create table t(a int, b int)") @@ -1199,24 +1219,28 @@ func (s *testSuiteJoin1) TestIssue5278(c *C) { tk.MustQuery("select * from t left join tt on t.a=tt.a left join t ttt on t.a=ttt.a").Check(testkit.Rows("1 1 1 1")) } -func (s *testSuiteJoin1) TestIssue15850JoinNullValue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15850JoinNullValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustQuery("SELECT * FROM (select null) v NATURAL LEFT JOIN (select null) v1;").Check(testkit.Rows("")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustExec("drop table if exists t0;") tk.MustExec("drop view if exists v0;") tk.MustExec("CREATE TABLE t0(c0 TEXT);") tk.MustExec("CREATE VIEW v0(c0) AS SELECT NULL;") tk.MustQuery("SELECT /*+ HASH_JOIN(v0) */ * FROM v0 NATURAL LEFT JOIN t0;").Check(testkit.Rows("")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("SELECT /*+ MERGE_JOIN(v0) */ * FROM v0 NATURAL LEFT JOIN t0;").Check(testkit.Rows("")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) } -func (s *testSuiteJoin1) TestIndexLookupJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexLookupJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("DROP TABLE IF EXISTS t") @@ -1403,13 +1427,15 @@ func (s *testSuiteJoin1) TestIndexLookupJoin(c *C) { tk.MustQuery("select /*+ inl_merge_join(t1)*/ * from t1 join t2 on t2.b=t1.id and t2.a=t1.id;").Check(testkit.Rows("1 1 1")) } -func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexNestedLoopHashJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("set @@tidb_index_join_batch_size=10") tk.MustExec("DROP TABLE IF EXISTS t, s") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table t(pk int primary key, a int)") for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i)) @@ -1434,13 +1460,13 @@ func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { )) rs := tk.MustQuery("select /*+ INL_HASH_JOIN(s) */ * from t left join s on t.a=s.a order by t.pk") for i, row := range rs.Rows() { - c.Assert(row[0].(string), Equals, fmt.Sprintf("%d", i)) + require.Equal(t, fmt.Sprintf("%d", i), row[0].(string)) } // index hash join with semi join - c.Assert(failpoint.Enable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin", "return(true)"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin", "return(true)")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/planner/core/MockOnlyEnableIndexHashJoin")) }() tk.MustExec("drop table t") tk.MustExec("CREATE TABLE `t` ( `l_orderkey` int(11) NOT NULL,`l_linenumber` int(11) NOT NULL,`l_partkey` int(11) DEFAULT NULL,`l_suppkey` int(11) DEFAULT NULL,PRIMARY KEY (`l_orderkey`,`l_linenumber`))") @@ -1457,8 +1483,8 @@ func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { tk.MustExec("analyze table t") // test semi join - tk.Se.GetSessionVars().InitChunkSize = 2 - tk.Se.GetSessionVars().MaxChunkSize = 2 + tk.Session().GetSessionVars().InitChunkSize = 2 + tk.Session().GetSessionVars().MaxChunkSize = 2 tk.MustExec("set @@tidb_index_join_batch_size=2") tk.MustQuery("desc format = 'brief' select * from t l1 where exists ( select * from t l2 where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey <> l1.l_suppkey ) order by `l_orderkey`,`l_linenumber`;").Check(testkit.Rows( "Sort 7.20 root test.t.l_orderkey, test.t.l_linenumber", @@ -1541,8 +1567,10 @@ func (s *testSuiteJoinSerial) TestIndexNestedLoopHashJoin(c *C) { tk.MustExec("drop table orders") } -func (s *testSuiteJoin3) TestIssue15686(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15686(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, k;") tk.MustExec("create table k (a int, pk int primary key, index(a));") @@ -1554,8 +1582,10 @@ func (s *testSuiteJoin3) TestIssue15686(c *C) { tk.MustQuery("select /*+ inl_merge_join(t) */ count(*) from k left join t on k.a = t.a and k.pk > t.pk;").Check(testkit.Rows("33")) } -func (s *testSuiteJoin3) TestIssue13449(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue13449(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, s;") tk.MustExec("create table t(a int, index(a));") @@ -1578,8 +1608,10 @@ func (s *testSuiteJoin3) TestIssue13449(c *C) { tk.MustQuery("select /*+ INL_HASH_JOIN(s) */ * from t join s on t.a=s.a order by t.a;").Check(testkit.Rows("1 1", "128 128")) } -func (s *testSuiteJoin3) TestMergejoinOrder(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMergejoinOrder(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1(a bigint primary key, b bigint);") @@ -1623,8 +1655,10 @@ func (s *testSuiteJoin3) TestMergejoinOrder(c *C) { )) } -func (s *testSuiteJoin1) TestEmbeddedOuterJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEmbeddedOuterJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") @@ -1634,8 +1668,10 @@ func (s *testSuiteJoin1) TestEmbeddedOuterJoin(c *C) { Check(testkit.Rows("1 1 ")) } -func (s *testSuiteJoin1) TestHashJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int);") @@ -1654,16 +1690,18 @@ func (s *testSuiteJoin1) TestHashJoin(c *C) { // 5 └─Selection_11 9990.00 5 cop[tikv] time:243.395µs, loops:6 not(isnull(test.t1.a)) N/A N/A // 6 └─TableFullScan_10 10000.00 5 cop[tikv] table:t1 time:206.273µs, loops:6 keep order:false, stats:pseudo N/A N/A row := result.Rows() - c.Assert(len(row), Equals, 7) + require.Equal(t, 7, len(row)) innerActRows := row[1][2].(string) - c.Assert(innerActRows, Equals, "0") + require.Equal(t, "0", innerActRows) outerActRows := row[4][2].(string) // FIXME: revert this result to 1 after TableReaderExecutor can handle initChunkSize. - c.Assert(outerActRows, Equals, "5") + require.Equal(t, "5", outerActRows) } -func (s *testSuiteJoin1) TestJoinDifferentDecimals(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinDifferentDecimals(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("Use test") tk.MustExec("Drop table if exists t1") tk.MustExec("Create table t1 (v int)") @@ -1677,12 +1715,14 @@ func (s *testSuiteJoin1) TestJoinDifferentDecimals(c *C) { tk.MustExec("Insert into t2 value (000003.000000)") rst := tk.MustQuery("Select * from t1, t2 where t1.v = t2.v order by t1.v") row := rst.Rows() - c.Assert(len(row), Equals, 3) + require.Equal(t, 3, len(row)) rst.Check(testkit.Rows("1 1.000", "2 2.000", "3 3.000")) } -func (s *testSuiteJoin2) TestNullEmptyAwareSemiJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNullEmptyAwareSemiJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, index idx_a(a), index idb_b(b), index idx_c(c))") @@ -2075,8 +2115,10 @@ func (s *testSuiteJoin2) TestNullEmptyAwareSemiJoin(c *C) { )) } -func (s *testSuiteJoin1) TestScalarFuncNullSemiJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestScalarFuncNullSemiJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") @@ -2091,8 +2133,10 @@ func (s *testSuiteJoin1) TestScalarFuncNullSemiJoin(c *C) { tk.MustQuery("select a in (select a+b from s) from t").Check(testkit.Rows("", "")) } -func (s *testSuiteJoin1) TestInjectProjOnTopN(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInjectProjOnTopN(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") @@ -2104,8 +2148,10 @@ func (s *testSuiteJoin1) TestInjectProjOnTopN(c *C) { )) } -func (s *testSuiteJoin1) TestIssue11544(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue11544(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table 11544t(a int)") tk.MustExec("create table 11544tt(a int, b varchar(10), index idx(a, b(3)))") @@ -2122,8 +2168,10 @@ func (s *testSuiteJoin1) TestIssue11544(c *C) { tk.MustQuery("select /*+ INL_MERGE_JOIN(tt) */ * from 11544t t, 11544tt tt where t.a=tt.a and tt.b in ('aaaaaaa', 'aaaabbb', 'aaaacccc')").Sort().Check(testkit.Rows("1 1 aaaaaaa", "1 1 aaaabbb", "1 1 aaaacccc")) } -func (s *testSuiteJoin1) TestIssue11390(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue11390(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table 11390t (k1 int unsigned, k2 int unsigned, key(k1, k2))") tk.MustExec("insert into 11390t values(1, 1)") @@ -2132,10 +2180,12 @@ func (s *testSuiteJoin1) TestIssue11390(c *C) { tk.MustQuery("select /*+ INL_MERGE_JOIN(t1, t2) */ * from 11390t t1, 11390t t2 where t1.k2 > 0 and t1.k2 = t2.k2 and t2.k1=1;").Check(testkit.Rows("1 1 1 1")) } -func (s *testSuiteJoinSerial) TestOuterTableBuildHashTableIsuse13933(c *C) { +func TestOuterTableBuildHashTableIsuse13933(t *testing.T) { plannercore.ForceUseOuterBuild4Test = true defer func() { plannercore.ForceUseOuterBuild4Test = false }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, s") tk.MustExec("create table t (a int,b int)") @@ -2167,8 +2217,10 @@ func (s *testSuiteJoinSerial) TestOuterTableBuildHashTableIsuse13933(c *C) { " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo")) } -func (s *testSuiteJoin1) TestIssue13177(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue13177(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a varchar(20), b int, c int)") @@ -2207,22 +2259,26 @@ func (s *testSuiteJoin1) TestIssue13177(c *C) { )) } -func (s *testSuiteJoin1) TestIssue14514(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue14514(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (pk varchar(14) primary key, a varchar(12));") tk.MustQuery("select * from (select t1.pk or '/' as c from t as t1 left join t as t2 on t1.a = t2.pk) as t where t.c = 1;").Check(testkit.Rows()) } -func (s *testSuiteJoinSerial) TestOuterMatchStatusIssue14742(c *C) { +func TestOuterMatchStatusIssue14742(t *testing.T) { plannercore.ForceUseOuterBuild4Test = true defer func() { plannercore.ForceUseOuterBuild4Test = false }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists testjoin;") tk.MustExec("create table testjoin(a int);") - tk.Se.GetSessionVars().MaxChunkSize = 2 + tk.Session().GetSessionVars().MaxChunkSize = 2 tk.MustExec("insert into testjoin values (NULL);") tk.MustExec("insert into testjoin values (1);") @@ -2238,13 +2294,15 @@ func (s *testSuiteJoinSerial) TestOuterMatchStatusIssue14742(c *C) { )) } -func (s *testSuiteJoinSerial) TestInlineProjection4HashJoinIssue15316(c *C) { +func TestInlineProjection4HashJoinIssue15316(t *testing.T) { // Two necessary factors to reproduce this issue: // (1) taking HashLeftJoin, i.e., letting the probing tuple lay at the left side of joined tuples // (2) the projection only contains a part of columns from the build side, i.e., pruning the same probe side plannercore.ForcedHashLeftJoin4Test = true defer func() { plannercore.ForcedHashLeftJoin4Test = false }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists S, T") tk.MustExec("create table S (a int not null, b int, c int);") @@ -2272,16 +2330,15 @@ func (s *testSuiteJoinSerial) TestInlineProjection4HashJoinIssue15316(c *C) { " └─TableFullScan 10000.00 cop[tikv] table:S keep order:false, stats:pseudo")) } -func (s *testSuiteJoinSerial) TestIssue18070(c *C) { +func TestIssue18070(t *testing.T) { + restoreFunc := config.RestoreFunc() + defer restoreFunc() config.UpdateGlobal(func(conf *config.Config) { conf.OOMAction = config.OOMActionCancel }) - defer func() { - config.UpdateGlobal(func(conf *config.Config) { - conf.OOMAction = config.OOMActionLog - }) - }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, index(a))") @@ -2290,19 +2347,21 @@ func (s *testSuiteJoinSerial) TestIssue18070(c *C) { tk.MustExec("insert into t2 values(1),(1),(2),(2)") tk.MustExec("set @@tidb_mem_quota_query=1000") err := tk.QueryToErr("select /*+ inl_hash_join(t1)*/ * from t1 join t2 on t1.a = t2.a;") - c.Assert(strings.Contains(err.Error(), "Out Of Memory Quota!"), IsTrue) + require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!")) fpName := "github.com/pingcap/tidb/executor/mockIndexMergeJoinOOMPanic" - c.Assert(failpoint.Enable(fpName, `panic("ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]")`), IsNil) + require.NoError(t, failpoint.Enable(fpName, `panic("ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]")`)) defer func() { - c.Assert(failpoint.Disable(fpName), IsNil) + require.NoError(t, failpoint.Disable(fpName)) }() err = tk.QueryToErr("select /*+ inl_merge_join(t1)*/ * from t1 join t2 on t1.a = t2.a;") - c.Assert(strings.Contains(err.Error(), "Out Of Memory Quota!"), IsTrue) + require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!")) } -func (s *testSuiteJoin1) TestIssue18564(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue18564(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, primary key(a), index idx(b,a));") @@ -2313,8 +2372,10 @@ func (s *testSuiteJoin1) TestIssue18564(c *C) { } // test hash join when enum column got invalid value -func (s *testSuiteJoin1) TestInvalidEnumVal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInvalidEnumVal(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set sql_mode = '';") tk.MustExec("drop table if exists t1;") @@ -2327,65 +2388,77 @@ func (s *testSuiteJoin1) TestInvalidEnumVal(c *C) { rows.Check(testkit.Rows("a a", " ", " ", " ", " ")) } -func (s *testSuiteJoinSerial) TestIssue18572_1(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue18572_1(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(a int, b int, index idx(b));") tk.MustExec("insert into t1 values(1, 1);") tk.MustExec("insert into t1 select * from t1;") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinInnerWorkerErr", "return"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinInnerWorkerErr", "return")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinInnerWorkerErr"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinInnerWorkerErr")) }() rs, err := tk.Exec("select /*+ inl_hash_join(t1) */ * from t1 right join t1 t2 on t1.b=t2.b;") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = session.GetRows4Test(context.Background(), nil, rs) - c.Assert(strings.Contains(err.Error(), "mockIndexHashJoinInnerWorkerErr"), IsTrue) - c.Assert(rs.Close(), IsNil) + require.True(t, strings.Contains(err.Error(), "mockIndexHashJoinInnerWorkerErr")) + require.NoError(t, rs.Close()) } -func (s *testSuiteJoinSerial) TestIssue18572_2(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue18572_2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(a int, b int, index idx(b));") tk.MustExec("insert into t1 values(1, 1);") tk.MustExec("insert into t1 select * from t1;") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinOuterWorkerErr", "return"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinOuterWorkerErr", "return")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinOuterWorkerErr"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinOuterWorkerErr")) }() rs, err := tk.Exec("select /*+ inl_hash_join(t1) */ * from t1 right join t1 t2 on t1.b=t2.b;") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = session.GetRows4Test(context.Background(), nil, rs) - c.Assert(strings.Contains(err.Error(), "mockIndexHashJoinOuterWorkerErr"), IsTrue) - c.Assert(rs.Close(), IsNil) + require.True(t, strings.Contains(err.Error(), "mockIndexHashJoinOuterWorkerErr")) + require.NoError(t, rs.Close()) } -func (s *testSuiteJoinSerial) TestIssue18572_3(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue18572_3(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(a int, b int, index idx(b));") tk.MustExec("insert into t1 values(1, 1);") tk.MustExec("insert into t1 select * from t1;") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinBuildErr", "return"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIndexHashJoinBuildErr", "return")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinBuildErr"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIndexHashJoinBuildErr")) }() rs, err := tk.Exec("select /*+ inl_hash_join(t1) */ * from t1 right join t1 t2 on t1.b=t2.b;") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = session.GetRows4Test(context.Background(), nil, rs) - c.Assert(strings.Contains(err.Error(), "mockIndexHashJoinBuildErr"), IsTrue) - c.Assert(rs.Close(), IsNil) + require.True(t, strings.Contains(err.Error(), "mockIndexHashJoinBuildErr")) + require.NoError(t, rs.Close()) } -func (s *testSuite9) TestApplyOuterAggEmptyInput(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestApplyOuterAggEmptyInput(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int)") tk.MustExec("create table t2(a int)") @@ -2405,8 +2478,11 @@ func (s *testSuite9) TestApplyOuterAggEmptyInput(c *C) { )) } -func (s *testSuite9) TestIssue19112(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue19112(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 ( c_int int, c_decimal decimal(12, 6), key(c_int), unique key(c_decimal) )") tk.MustExec("create table t2 like t1") @@ -2417,8 +2493,11 @@ func (s *testSuite9) TestIssue19112(c *C) { "3 1.010000 3 1.010000")) } -func (s *testSuiteJoin3) TestIssue11896(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue11896(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") // compare bigint to bit(64) tk.MustExec("drop table if exists t") @@ -2471,8 +2550,11 @@ func (s *testSuiteJoin3) TestIssue11896(c *C) { testkit.Rows("1 \x01")) } -func (s *testSuiteJoin3) TestIssue19498(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue19498(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1 (c_int int, primary key (c_int));") @@ -2501,8 +2583,11 @@ func (s *testSuiteJoin3) TestIssue19498(c *C) { rs.Check(testkit.Rows("happy archimedes 2", "happy fermat 2", "happy hypatia 2", "zen sammet 4")) } -func (s *testSuiteJoin3) TestIssue19500(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue19500(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1 (c_int int, primary key (c_int));") tk.MustExec("insert into t1 values (1),(2),(3),(4),(5);") @@ -2516,8 +2601,11 @@ func (s *testSuiteJoin3) TestIssue19500(c *C) { Check(testkit.Rows("", "", "3", "3", "3")) } -func (s *testSuiteJoinSerial) TestExplainAnalyzeJoin(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestExplainAnalyzeJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2;") tk.MustExec("create table t1 (a int, b int, unique index (a));") tk.MustExec("create table t2 (a int, b int, unique index (a))") @@ -2525,55 +2613,59 @@ func (s *testSuiteJoinSerial) TestExplainAnalyzeJoin(c *C) { tk.MustExec("insert into t2 values (1,1),(2,2),(3,3),(4,4),(5,5)") // Test for index lookup join. rows := tk.MustQuery("explain analyze select /*+ INL_JOIN(t1, t2) */ * from t1,t2 where t1.a=t2.a;").Rows() - c.Assert(len(rows), Equals, 8) - c.Assert(rows[0][0], Matches, "IndexJoin_.*") - c.Assert(rows[0][5], Matches, "time:.*, loops:.*, inner:{total:.*, concurrency:.*, task:.*, construct:.*, fetch:.*, build:.*}, probe:.*") + require.Equal(t, 8, len(rows)) + require.Regexp(t, "IndexJoin_.*", rows[0][0]) + require.Regexp(t, "time:.*, loops:.*, inner:{total:.*, concurrency:.*, task:.*, construct:.*, fetch:.*, build:.*}, probe:.*", rows[0][5]) // Test for index lookup hash join. rows = tk.MustQuery("explain analyze select /*+ INL_HASH_JOIN(t1, t2) */ * from t1,t2 where t1.a=t2.a;").Rows() - c.Assert(len(rows), Equals, 8) - c.Assert(rows[0][0], Matches, "IndexHashJoin.*") - c.Assert(rows[0][5], Matches, "time:.*, loops:.*, inner:{total:.*, concurrency:.*, task:.*, construct:.*, fetch:.*, build:.*, join:.*}") + require.Equal(t, 8, len(rows)) + require.Regexp(t, "IndexHashJoin.*", rows[0][0]) + require.Regexp(t, "time:.*, loops:.*, inner:{total:.*, concurrency:.*, task:.*, construct:.*, fetch:.*, build:.*, join:.*}", rows[0][5]) // Test for hash join. rows = tk.MustQuery("explain analyze select /*+ HASH_JOIN(t1, t2) */ * from t1,t2 where t1.a=t2.a;").Rows() - c.Assert(len(rows), Equals, 7) - c.Assert(rows[0][0], Matches, "HashJoin.*") - c.Assert(rows[0][5], Matches, "time:.*, loops:.*, build_hash_table:{total:.*, fetch:.*, build:.*}, probe:{concurrency:5, total:.*, max:.*, probe:.*, fetch:.*}") + require.Equal(t, 7, len(rows)) + require.Regexp(t, "HashJoin.*", rows[0][0]) + require.Regexp(t, "time:.*, loops:.*, build_hash_table:{total:.*, fetch:.*, build:.*}, probe:{concurrency:5, total:.*, max:.*, probe:.*, fetch:.*}", rows[0][5]) // Test for index merge join. rows = tk.MustQuery("explain analyze select /*+ INL_MERGE_JOIN(t1, t2) */ * from t1,t2 where t1.a=t2.a;").Rows() - c.Assert(len(rows), Equals, 9) - c.Assert(rows[0][0], Matches, "IndexMergeJoin_.*") - c.Assert(rows[0][5], Matches, fmt.Sprintf(".*Concurrency:%v.*", tk.Se.GetSessionVars().IndexLookupJoinConcurrency())) + require.Len(t, rows, 9) + require.Regexp(t, "IndexMergeJoin_.*", rows[0][0]) + require.Regexp(t, fmt.Sprintf(".*Concurrency:%v.*", tk.Session().GetSessionVars().IndexLookupJoinConcurrency()), rows[0][5]) } -func (s *testSuiteJoinSerial) TestIssue20270(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - err := failpoint.Enable("github.com/pingcap/tidb/executor/killedInJoin2Chunk", "return(true)") - c.Assert(err, IsNil) +func TestIssue20270(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/killedInJoin2Chunk", "return(true)")) tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t(c1 int, c2 int)") tk.MustExec("create table t1(c1 int, c2 int)") tk.MustExec("insert into t values(1,1),(2,2)") tk.MustExec("insert into t1 values(2,3),(4,4)") - err = tk.QueryToErr("select /*+ TIDB_HJ(t, t1) */ * from t left join t1 on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20") - c.Assert(err, Equals, executor.ErrQueryInterrupted) - err = failpoint.Disable("github.com/pingcap/tidb/executor/killedInJoin2Chunk") - c.Assert(err, IsNil) + err := tk.QueryToErr("select /*+ TIDB_HJ(t, t1) */ * from t left join t1 on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20") + require.Equal(t, executor.ErrQueryInterrupted, err) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/killedInJoin2Chunk")) plannercore.ForceUseOuterBuild4Test = true defer func() { plannercore.ForceUseOuterBuild4Test = false }() err = failpoint.Enable("github.com/pingcap/tidb/executor/killedInJoin2ChunkForOuterHashJoin", "return(true)") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("insert into t1 values(1,30),(2,40)") err = tk.QueryToErr("select /*+ TIDB_HJ(t, t1) */ * from t left outer join t1 on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20") - c.Assert(err, Equals, executor.ErrQueryInterrupted) + require.Equal(t, executor.ErrQueryInterrupted, err) err = failpoint.Disable("github.com/pingcap/tidb/executor/killedInJoin2ChunkForOuterHashJoin") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testSuiteJoinSerial) TestIssue20710(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20710(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists s;") tk.MustExec("create table t(a int, b int)") @@ -2595,27 +2687,33 @@ func (s *testSuiteJoinSerial) TestIssue20710(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows()) } -func (s *testSuiteJoinSerial) TestIssue20779(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20779(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(a int, b int, index idx(b));") tk.MustExec("insert into t1 values(1, 1);") tk.MustExec("insert into t1 select * from t1;") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testIssue20779", "return"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testIssue20779", "return")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testIssue20779"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testIssue20779")) }() rs, err := tk.Exec("select /*+ inl_hash_join(t2) */ t1.b from t1 left join t1 t2 on t1.b=t2.b order by t1.b;") - c.Assert(err, IsNil) + require.NoError(t, err) _, err = session.GetRows4Test(context.Background(), nil, rs) - c.Assert(err.Error(), Matches, "testIssue20779") - c.Assert(rs.Close(), IsNil) + require.EqualError(t, err, "testIssue20779") + require.NoError(t, rs.Close()) } -func (s *testSuiteJoinSerial) TestIssue20219(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20219(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t,s ") tk.MustExec("CREATE TABLE `t` ( `a` set('a','b','c','d','e','f','g','h','i','j') DEFAULT NULL );") tk.MustExec("insert into t values('i'), ('j');") @@ -2627,8 +2725,11 @@ func (s *testSuiteJoinSerial) TestIssue20219(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows()) } -func (s *testSuiteJoinSerial) TestIssue25902(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue25902(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists tt1,tt2,tt3; ") tk.MustExec("create table tt1 (ts timestamp);") tk.MustExec("create table tt2 (ts varchar(32));") @@ -2644,23 +2745,26 @@ func (s *testSuiteJoinSerial) TestIssue25902(c *C) { tk.MustExec("set @@session.time_zone = @tmp;") } -func (s *testSuiteJoinSerial) TestIssue30211(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue30211(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1(a int, index(a));") tk.MustExec("create table t2(a int, index(a));") func() { fpName := "github.com/pingcap/tidb/executor/TestIssue30211" - c.Assert(failpoint.Enable(fpName, `panic("TestIssue30211 IndexJoinPanic")`), IsNil) + require.NoError(t, failpoint.Enable(fpName, `panic("TestIssue30211 IndexJoinPanic")`)) defer func() { - c.Assert(failpoint.Disable(fpName), IsNil) + require.NoError(t, failpoint.Disable(fpName)) }() - err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error() - c.Assert(err, Matches, "failpoint panic: TestIssue30211 IndexJoinPanic") + err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.a;") + require.EqualError(t, err, "failpoint panic: TestIssue30211 IndexJoinPanic") + + err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 join t2 on t1.a = t2.a;") + require.EqualError(t, err, "failpoint panic: TestIssue30211 IndexJoinPanic") - err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error() - c.Assert(err, Matches, "failpoint panic: TestIssue30211 IndexJoinPanic") }() tk.MustExec("insert into t1 values(1),(2);") tk.MustExec("insert into t2 values(1),(1),(2),(2);") @@ -2675,18 +2779,20 @@ func (s *testSuiteJoinSerial) TestIssue30211(c *C) { }) }() err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error() - c.Assert(strings.Contains(err, "Out Of Memory Quota"), IsTrue) + require.True(t, strings.Contains(err, "Out Of Memory Quota")) err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error() - c.Assert(strings.Contains(err, "Out Of Memory Quota"), IsTrue) + require.True(t, strings.Contains(err, "Out Of Memory Quota")) } -func (s *testSuiteJoinSerial) TestIssue31129(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue31129(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") tk.MustExec("set @@tidb_index_join_batch_size=10") tk.MustExec("DROP TABLE IF EXISTS t, s") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table t(pk int primary key, a int)") for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i, i)) @@ -2700,31 +2806,31 @@ func (s *testSuiteJoinSerial) TestIssue31129(c *C) { // Test IndexNestedLoopHashJoin keepOrder. fpName := "github.com/pingcap/tidb/executor/TestIssue31129" - c.Assert(failpoint.Enable(fpName, "return"), IsNil) + require.NoError(t, failpoint.Enable(fpName, "return")) err := tk.QueryToErr("select /*+ INL_HASH_JOIN(s) */ * from t left join s on t.a=s.a order by t.pk") - c.Assert(strings.Contains(err.Error(), "TestIssue31129"), IsTrue) - c.Assert(failpoint.Disable(fpName), IsNil) + require.True(t, strings.Contains(err.Error(), "TestIssue31129")) + require.NoError(t, failpoint.Disable(fpName)) // Test IndexNestedLoopHashJoin build hash table panic. fpName = "github.com/pingcap/tidb/executor/IndexHashJoinBuildHashTablePanic" - c.Assert(failpoint.Enable(fpName, `panic("IndexHashJoinBuildHashTablePanic")`), IsNil) + require.NoError(t, failpoint.Enable(fpName, `panic("IndexHashJoinBuildHashTablePanic")`)) err = tk.QueryToErr("select /*+ INL_HASH_JOIN(s) */ * from t left join s on t.a=s.a order by t.pk") - c.Assert(strings.Contains(err.Error(), "IndexHashJoinBuildHashTablePanic"), IsTrue) - c.Assert(failpoint.Disable(fpName), IsNil) + require.True(t, strings.Contains(err.Error(), "IndexHashJoinBuildHashTablePanic")) + require.NoError(t, failpoint.Disable(fpName)) // Test IndexNestedLoopHashJoin fetch inner fail. fpName = "github.com/pingcap/tidb/executor/IndexHashJoinFetchInnerResultsErr" - c.Assert(failpoint.Enable(fpName, "return"), IsNil) + require.NoError(t, failpoint.Enable(fpName, "return")) err = tk.QueryToErr("select /*+ INL_HASH_JOIN(s) */ * from t left join s on t.a=s.a order by t.pk") - c.Assert(strings.Contains(err.Error(), "IndexHashJoinFetchInnerResultsErr"), IsTrue) - c.Assert(failpoint.Disable(fpName), IsNil) + require.True(t, strings.Contains(err.Error(), "IndexHashJoinFetchInnerResultsErr")) + require.NoError(t, failpoint.Disable(fpName)) // Test IndexNestedLoopHashJoin build hash table panic and IndexNestedLoopHashJoin fetch inner fail at the same time. fpName1, fpName2 := "github.com/pingcap/tidb/executor/IndexHashJoinBuildHashTablePanic", "github.com/pingcap/tidb/executor/IndexHashJoinFetchInnerResultsErr" - c.Assert(failpoint.Enable(fpName1, `panic("IndexHashJoinBuildHashTablePanic")`), IsNil) - c.Assert(failpoint.Enable(fpName2, "return"), IsNil) + require.NoError(t, failpoint.Enable(fpName1, `panic("IndexHashJoinBuildHashTablePanic")`)) + require.NoError(t, failpoint.Enable(fpName2, "return")) err = tk.QueryToErr("select /*+ INL_HASH_JOIN(s) */ * from t left join s on t.a=s.a order by t.pk") - c.Assert(strings.Contains(err.Error(), "IndexHashJoinBuildHashTablePanic"), IsTrue) - c.Assert(failpoint.Disable(fpName1), IsNil) - c.Assert(failpoint.Disable(fpName2), IsNil) + require.True(t, strings.Contains(err.Error(), "IndexHashJoinBuildHashTablePanic")) + require.NoError(t, failpoint.Disable(fpName1)) + require.NoError(t, failpoint.Disable(fpName2)) } diff --git a/executor/joiner_test.go b/executor/joiner_test.go index 2630290fb78f2..ff87d271fdcfd 100644 --- a/executor/joiner_test.go +++ b/executor/joiner_test.go @@ -16,22 +16,16 @@ package executor import ( "math/rand" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testSuiteJoiner{}) - -type testSuiteJoiner struct{} - -func (s *testSuiteJoiner) SetUpSuite(c *C) { -} - -func (s *testSuiteJoiner) TestRequiredRows(c *C) { +func TestRequiredRows(t *testing.T) { joinTypes := []core.JoinType{core.InnerJoin, core.LeftOuterJoin, core.RightOuterJoin} lTypes := [][]byte{ {mysql.TypeLong}, @@ -74,8 +68,8 @@ func (s *testSuiteJoiner) TestRequiredRows(c *C) { it := chunk.NewIterator4Chunk(innerChk) it.Begin() _, _, err := joiner.tryToMatchInners(outerRow, it, result) - c.Assert(err, IsNil) - c.Assert(result.NumRows(), Equals, required) + require.NoError(t, err) + require.Equal(t, required, result.NumRows()) } } } diff --git a/executor/main_test.go b/executor/main_test.go index 88be5b84f0750..59757b1e1e513 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -16,7 +16,6 @@ package executor_test import ( "fmt" - "os" "testing" "github.com/pingcap/tidb/config" @@ -32,14 +31,16 @@ import ( var testDataMap = make(testdata.BookKeeper) var prepareMergeSuiteData testdata.TestData var aggMergeSuiteData testdata.TestData +var executorSuiteData testdata.TestData func TestMain(m *testing.M) { testbridge.SetupForCommonTest() - - testDataMap.LoadTestSuiteData("testdata", "prepare_suite") testDataMap.LoadTestSuiteData("testdata", "agg_suite") - prepareMergeSuiteData = testDataMap["prepare_suite"] + testDataMap.LoadTestSuiteData("testdata", "executor_suite") + testDataMap.LoadTestSuiteData("testdata", "prepare_suite") aggMergeSuiteData = testDataMap["agg_suite"] + executorSuiteData = testDataMap["executor_suite"] + prepareMergeSuiteData = testDataMap["prepare_suite"] autoid.SetStep(5000) config.UpdateGlobal(func(conf *config.Config) { @@ -49,14 +50,12 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) tikv.EnableFailpoints() - tmpDir := config.GetGlobalConfig().TempStoragePath - _ = os.RemoveAll(tmpDir) // clean the uncleared temp file during the last run. - _ = os.MkdirAll(tmpDir, 0755) opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/txnkv/transaction.keepAlive"), } callback := func(i int) int { testDataMap.GenerateOutputIfNeeded() diff --git a/executor/memory_test.go b/executor/memory_test.go index 12843f86079e8..b4fdbaeceae7e 100644 --- a/executor/memory_test.go +++ b/executor/memory_test.go @@ -18,112 +18,84 @@ import ( "context" "fmt" "runtime" + "runtime/debug" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testMemoryLeak{}) - -type testMemoryLeak struct { - store kv.Storage - domain *domain.Domain -} - -func (s *testMemoryLeak) SetUpSuite(c *C) { - var err error - s.store, err = mockstore.NewMockStore() - c.Assert(err, IsNil) - s.domain, err = session.BootstrapSession(s.store) - c.Assert(err, IsNil) -} - -func (s *testMemoryLeak) TearDownSuite(c *C) { - s.domain.Close() - c.Assert(s.store.Close(), IsNil) -} - -func (s *testMemoryLeak) TestPBMemoryLeak(c *C) { - c.Skip("too slow") - - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "create database test_mem") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test_mem") - c.Assert(err, IsNil) +func TestPBMemoryLeak(t *testing.T) { + debug.SetGCPercent(1000) + defer debug.SetGCPercent(100) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database test_mem") + tk.MustExec("use test_mem") // prepare data totalSize := uint64(256 << 20) // 256MB blockSize := uint64(8 << 10) // 8KB delta := totalSize / 5 numRows := totalSize / blockSize - _, err = se.Execute(context.Background(), fmt.Sprintf("create table t (c varchar(%v))", blockSize)) - c.Assert(err, IsNil) - defer func() { - _, err = se.Execute(context.Background(), "drop table t") - c.Assert(err, IsNil) - }() + tk.MustExec(fmt.Sprintf("create table t (c varchar(%v))", blockSize)) sql := fmt.Sprintf("insert into t values (space(%v))", blockSize) for i := uint64(0); i < numRows; i++ { - _, err = se.Execute(context.Background(), sql) - c.Assert(err, IsNil) + tk.MustExec(sql) } // read data runtime.GC() - allocatedBegin, inUseBegin := s.readMem() - records, err := se.Execute(context.Background(), "select * from t") - c.Assert(err, IsNil) + allocatedBegin, inUseBegin := readMem() + records, err := tk.Session().Execute(context.Background(), "select * from t") + require.NoError(t, err) record := records[0] rowCnt := 0 chk := record.NewChunk(nil) for { - c.Assert(record.Next(context.Background(), chk), IsNil) + require.NoError(t, record.Next(context.Background(), chk)) rowCnt += chk.NumRows() if chk.NumRows() == 0 { break } } - c.Assert(rowCnt, Equals, int(numRows)) + require.Equal(t, int(numRows), rowCnt) // check memory before close runtime.GC() - allocatedAfter, inUseAfter := s.readMem() - c.Assert(allocatedAfter-allocatedBegin, GreaterEqual, totalSize) - c.Assert(s.memDiff(inUseAfter, inUseBegin), Less, delta) + allocatedAfter, inUseAfter := readMem() + require.GreaterOrEqual(t, allocatedAfter-allocatedBegin, totalSize) + require.Less(t, memDiff(inUseAfter, inUseBegin), delta) - se.Close() runtime.GC() - allocatedFinal, inUseFinal := s.readMem() - c.Assert(allocatedFinal-allocatedAfter, Less, delta) - c.Assert(s.memDiff(inUseFinal, inUseAfter), Less, delta) + allocatedFinal, inUseFinal := readMem() + require.Less(t, allocatedFinal-allocatedAfter, delta) + require.Less(t, memDiff(inUseFinal, inUseAfter), delta) } // nolint:unused -func (s *testMemoryLeak) readMem() (allocated, heapInUse uint64) { +func readMem() (allocated, heapInUse uint64) { var stat runtime.MemStats runtime.ReadMemStats(&stat) return stat.TotalAlloc, stat.HeapInuse } // nolint:unused -func (s *testMemoryLeak) memDiff(m1, m2 uint64) uint64 { +func memDiff(m1, m2 uint64) uint64 { if m1 > m2 { return m1 - m2 } return m2 - m1 } -func (s *testMemoryLeak) TestGlobalMemoryTrackerOnCleanUp(c *C) { +func TestGlobalMemoryTrackerOnCleanUp(t *testing.T) { // TODO: assert the memory consume has happened in another way originConsume := executor.GlobalMemoryUsageTracker.BytesConsumed() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int)") @@ -133,12 +105,12 @@ func (s *testMemoryLeak) TestGlobalMemoryTrackerOnCleanUp(c *C) { tk.MustExec("insert t (id) values (2)") tk.MustExec("insert t (id) values (3)") afterConsume := executor.GlobalMemoryUsageTracker.BytesConsumed() - c.Assert(originConsume, Equals, afterConsume) + require.Equal(t, afterConsume, originConsume) // assert update tk.MustExec("update t set id = 4 where id = 1") tk.MustExec("update t set id = 5 where id = 2") tk.MustExec("update t set id = 6 where id = 3") afterConsume = executor.GlobalMemoryUsageTracker.BytesConsumed() - c.Assert(originConsume, Equals, afterConsume) + require.Equal(t, afterConsume, originConsume) } diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index cdb18b4096274..64729e3705dcb 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -984,3 +984,134 @@ func (e *hotRegionsHistoryRetriver) getHotRegionRowWithSchemaInfo( return rows, nil } + +type tikvRegionPeersRetriever struct { + dummyCloser + extractor *plannercore.TikvRegionPeersExtractor + retrieved bool +} + +func (e *tikvRegionPeersRetriever) retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error) { + if e.extractor.SkipRequest || e.retrieved { + return nil, nil + } + e.retrieved = true + tikvStore, ok := sctx.GetStore().(helper.Storage) + if !ok { + return nil, errors.New("Information about hot region can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + + var regionsInfo, regionsInfoByStoreID []helper.RegionInfo + regionMap := make(map[int64]*helper.RegionInfo) + storeMap := make(map[int64]struct{}) + + if len(e.extractor.StoreIDs) == 0 && len(e.extractor.RegionIDs) == 0 { + regionsInfo, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + return e.packTiKVRegionPeersRows(regionsInfo.Regions, storeMap) + } + + for _, storeID := range e.extractor.StoreIDs { + // if a region_id located in 1, 4, 7 store we will get all of them when request any store_id, + // storeMap is used to filter peers on unexpected stores. + storeMap[int64(storeID)] = struct{}{} + storeRegionsInfo, err := tikvHelper.GetStoreRegionsInfo(storeID) + if err != nil { + return nil, err + } + for i, regionInfo := range storeRegionsInfo.Regions { + // regionMap is used to remove dup regions and record the region in regionsInfoByStoreID. + if _, ok := regionMap[regionInfo.ID]; !ok { + regionsInfoByStoreID = append(regionsInfoByStoreID, regionInfo) + regionMap[regionInfo.ID] = &storeRegionsInfo.Regions[i] + } + } + } + + if len(e.extractor.RegionIDs) == 0 { + return e.packTiKVRegionPeersRows(regionsInfoByStoreID, storeMap) + } + + for _, regionID := range e.extractor.RegionIDs { + regionInfoByStoreID, ok := regionMap[int64(regionID)] + if !ok { + // if there is storeIDs, target region_id is fetched by storeIDs, + // otherwise we need to fetch it from PD. + if len(e.extractor.StoreIDs) == 0 { + regionInfo, err := tikvHelper.GetRegionInfoByID(regionID) + if err != nil { + return nil, err + } + regionsInfo = append(regionsInfo, *regionInfo) + } + } else { + regionsInfo = append(regionsInfo, *regionInfoByStoreID) + } + } + + return e.packTiKVRegionPeersRows(regionsInfo, storeMap) +} + +func (e *tikvRegionPeersRetriever) isUnexpectedStoreID(storeID int64, storeMap map[int64]struct{}) bool { + if len(e.extractor.StoreIDs) == 0 { + return false + } + if _, ok := storeMap[storeID]; ok { + return false + } + return true +} + +func (e *tikvRegionPeersRetriever) packTiKVRegionPeersRows( + regionsInfo []helper.RegionInfo, storeMap map[int64]struct{}) ([][]types.Datum, error) { + var rows [][]types.Datum + for _, region := range regionsInfo { + records := make([][]types.Datum, 0, len(region.Peers)) + pendingPeerIDSet := set.NewInt64Set() + for _, peer := range region.PendingPeers { + pendingPeerIDSet.Insert(peer.ID) + } + downPeerMap := make(map[int64]int64, len(region.DownPeers)) + for _, peerStat := range region.DownPeers { + downPeerMap[peerStat.Peer.ID] = peerStat.DownSec + } + for _, peer := range region.Peers { + // isUnexpectedStoreID return true if we should filter this peer. + if e.isUnexpectedStoreID(peer.StoreID, storeMap) { + continue + } + + row := make([]types.Datum, len(infoschema.TableTiKVRegionPeersCols)) + row[0].SetInt64(region.ID) + row[1].SetInt64(peer.ID) + row[2].SetInt64(peer.StoreID) + if peer.IsLearner { + row[3].SetInt64(1) + } else { + row[3].SetInt64(0) + } + if peer.ID == region.Leader.ID { + row[4].SetInt64(1) + } else { + row[4].SetInt64(0) + } + if downSec, ok := downPeerMap[peer.ID]; ok { + row[5].SetString(downPeer, mysql.DefaultCollationName) + row[6].SetInt64(downSec) + } else if pendingPeerIDSet.Exist(peer.ID) { + row[5].SetString(pendingPeer, mysql.DefaultCollationName) + } else { + row[5].SetString(normalPeer, mysql.DefaultCollationName) + } + records = append(records, row) + } + rows = append(rows, records...) + } + return rows, nil +} diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index 466068424937a..8439574f0b98d 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -26,6 +26,7 @@ import ( "net/http/httptest" "os" "path/filepath" + "strconv" "strings" "sync/atomic" "time" @@ -471,8 +472,7 @@ func (s *testClusterTableBase) setupClusterGRPCServer(c *C) map[string]*testServ // create gRPC servers for _, typ := range []string{"tidb", "tikv", "pd"} { - tmpDir, err := os.MkdirTemp("", typ) - c.Assert(err, IsNil) + tmpDir := c.MkDir() server := grpc.NewServer() logFile := filepath.Join(tmpDir, fmt.Sprintf("%s.log", typ)) @@ -1003,6 +1003,16 @@ func (s *testHotRegionsHistoryTableSuite) SetUpSuite(c *C) { s.startTime = time.Now() } +func writeResp(w http.ResponseWriter, resp interface{}) { + w.WriteHeader(http.StatusOK) + jsonResp, err := json.Marshal(resp) + if err != nil { + writeJSONError(w, http.StatusInternalServerError, "unable to marshal resp", err) + return + } + w.Write(jsonResp) +} + func writeJSONError(w http.ResponseWriter, code int, prefix string, err error) { type errorResponse struct { Error string `json:"error"` @@ -1428,3 +1438,190 @@ func (s *testHotRegionsHistoryTableSuite) TestTiDBHotRegionsHistoryError(c *C) { c.Assert(err.Error(), Equals, "denied to scan hot regions, please specified the end time, such as `update_time < '2020-01-01 00:00:00'`") c.Assert(rs.Close(), IsNil) } + +var regionsInfo = map[uint64]helper.RegionInfo{ + 1: { + ID: 1, + Peers: []helper.RegionPeer{{ID: 11, StoreID: 1, IsLearner: false}, {ID: 12, StoreID: 2, IsLearner: false}, {ID: 13, StoreID: 3, IsLearner: false}}, + Leader: helper.RegionPeer{ID: 11, StoreID: 1, IsLearner: false}, + }, + 2: { + ID: 2, + Peers: []helper.RegionPeer{{ID: 21, StoreID: 1, IsLearner: false}, {ID: 22, StoreID: 2, IsLearner: false}, {ID: 23, StoreID: 3, IsLearner: false}}, + Leader: helper.RegionPeer{ID: 22, StoreID: 2, IsLearner: false}, + }, + 3: { + ID: 3, + Peers: []helper.RegionPeer{{ID: 31, StoreID: 1, IsLearner: false}, {ID: 32, StoreID: 2, IsLearner: false}, {ID: 33, StoreID: 3, IsLearner: false}}, + Leader: helper.RegionPeer{ID: 33, StoreID: 3, IsLearner: false}, + }, +} + +var storeRegionsInfo = &helper.RegionsInfo{ + Count: 3, + Regions: []helper.RegionInfo{ + regionsInfo[1], + regionsInfo[2], + regionsInfo[3], + }, +} + +var storesRegionsInfo = map[uint64]*helper.RegionsInfo{ + 1: storeRegionsInfo, + 2: storeRegionsInfo, + 3: storeRegionsInfo, +} + +var _ = SerialSuites(&testTikvRegionPeersTableSuite{testInfoschemaTableSuiteBase: &testInfoschemaTableSuiteBase{}}) + +type testTikvRegionPeersTableSuite struct { + *testInfoschemaTableSuiteBase + httpServer *httptest.Server + mockAddr string + startTime time.Time +} + +func (s *testTikvRegionPeersTableSuite) SetUpSuite(c *C) { + s.testInfoschemaTableSuiteBase.SetUpSuite(c) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() +} + +func storesRegionsInfoHandler(w http.ResponseWriter, r *http.Request) { + vars := mux.Vars(r) + id, err := strconv.Atoi(vars["id"]) + if err != nil { + writeJSONError(w, http.StatusBadRequest, "unable to parse id", err) + return + } + writeResp(w, storesRegionsInfo[uint64(id)]) +} + +func regionsInfoHandler(w http.ResponseWriter, r *http.Request) { + vars := mux.Vars(r) + id, err := strconv.Atoi(vars["id"]) + if err != nil { + writeJSONError(w, http.StatusBadRequest, "unable to parse id", err) + return + } + writeResp(w, regionsInfo[uint64(id)]) +} +func (s *testTikvRegionPeersTableSuite) TearDownSuite(c *C) { + s.httpServer.Close() + s.testInfoschemaTableSuiteBase.TearDownSuite(c) +} + +func (s *testTikvRegionPeersTableSuite) setUpMockPDHTTPServer() (*httptest.Server, string) { + // mock PD http server + router := mux.NewRouter() + server := httptest.NewServer(router) + // mock store stats stat + mockAddr := strings.TrimPrefix(server.URL, "http://") + // mock PD API + router.Handle(pdapi.Status, fn.Wrap(func() (interface{}, error) { + return struct { + Version string `json:"version"` + GitHash string `json:"git_hash"` + StartTimestamp int64 `json:"start_timestamp"` + }{ + Version: "4.0.0-alpha", + GitHash: "mock-pd-githash", + StartTimestamp: s.startTime.Unix(), + }, nil + })) + // mock get regionsInfo by store id + router.HandleFunc(pdapi.StoreRegions+"/"+"{id}", storesRegionsInfoHandler) + // mock get regionInfo by region id + router.HandleFunc(pdapi.RegionByID+"/"+"{id}", regionsInfoHandler) + return server, mockAddr +} + +func (s *testTikvRegionPeersTableSuite) TestTikvRegionPeers(c *C) { + mockAddr := s.mockAddr + store := &mockStore{ + s.store.(helper.Storage), + mockAddr, + } + + fullRegionPeers := [][]string{ + {"1", "11", "1", "0", "1", "NORMAL", ""}, + {"1", "12", "2", "0", "0", "NORMAL", ""}, + {"1", "13", "3", "0", "0", "NORMAL", ""}, + + {"2", "21", "1", "0", "0", "NORMAL", ""}, + {"2", "22", "2", "0", "1", "NORMAL", ""}, + {"2", "23", "3", "0", "0", "NORMAL", ""}, + + {"3", "31", "1", "0", "0", "NORMAL", ""}, + {"3", "32", "2", "0", "0", "NORMAL", ""}, + {"3", "33", "3", "0", "1", "NORMAL", ""}, + } + + var cases = []struct { + conditions []string + reqCount int32 + expected [][]string + }{ + { + conditions: []string{ + "store_id in (1,2,3)", + "region_id in (1,2,3)", + }, + expected: fullRegionPeers, + }, + { + conditions: []string{ + "store_id in (1,2)", + "region_id=1", + }, + expected: [][]string{ + fullRegionPeers[0], fullRegionPeers[1], + }, + }, + { + conditions: []string{ + "store_id in (1,2)", + "region_id=1", + "is_leader=1", + }, + expected: [][]string{ + fullRegionPeers[0], + }, + }, + { + conditions: []string{ + "store_id in (1,2)", + "region_id=1", + "is_leader=0", + }, + expected: [][]string{ + fullRegionPeers[1], + }, + }, + { + conditions: []string{ + "store_id =1", + "region_id =1", + "is_leader =0", + }, + expected: [][]string{}, + }, + } + + tk := testkit.NewTestKit(c, store) + for _, cas := range cases { + sql := "select * from information_schema.tikv_region_peers" + if len(cas.conditions) > 0 { + sql = fmt.Sprintf("%s where %s", sql, strings.Join(cas.conditions, " and ")) + } + result := tk.MustQuery(sql) + warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + c.Assert(len(warnings), Equals, 0, Commentf("unexpected warnigns: %+v", warnings)) + var expected []string + for _, row := range cas.expected { + expectedRow := row + expected = append(expected, strings.Join(expectedRow, " ")) + } + result.Check(testkit.Rows(expected...)) + } +} diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 4d616513963e4..7b2c65a3e2d55 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -726,6 +726,10 @@ func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) { )) } +type testSuiteJoin3 struct { + *baseTestSuite +} + // TestVectorizedMergeJoin is used to test vectorized merge join with some corner cases. //nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { diff --git a/executor/metrics_reader.go b/executor/metrics_reader.go index d5bb294a212e3..3e90897d03192 100644 --- a/executor/metrics_reader.go +++ b/executor/metrics_reader.go @@ -233,11 +233,7 @@ func (e *MetricsSummaryRetriever) retrieve(ctx context.Context, sctx sessionctx. } exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, sql) - if err != nil { - return nil, errors.Errorf("execute '%s' failed: %v", sql, err) - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql) if err != nil { return nil, errors.Errorf("execute '%s' failed: %v", sql, err) } @@ -318,11 +314,7 @@ func (e *MetricsSummaryByLabelRetriever) retrieve(ctx context.Context, sctx sess util.MetricSchemaName.L, name, cond) } exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, sql) - if err != nil { - return nil, errors.Errorf("execute '%s' failed: %v", sql, err) - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, sql) if err != nil { return nil, errors.Errorf("execute '%s' failed: %v", sql, err) } diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index bdd6a2bfc8418..46b2bb92666dd 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -17,17 +17,20 @@ package executor_test import ( "context" "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite7) TestStmtLabel(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStmtLabel(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table label (c1 int primary key, c2 int, c3 int, index (c2))") for i := 0; i < 10; i++ { @@ -60,13 +63,12 @@ func (s *testSuite7) TestStmtLabel(c *C) { } for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") - c.Check(err, IsNil) + require.NoError(t, err) preprocessorReturn := &plannercore.PreprocessorReturn{} - err = plannercore.Preprocess(tk.Se, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) - c.Check(err, IsNil) - c.Assert(err, IsNil) - _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, preprocessorReturn.InfoSchema) - c.Assert(err, IsNil) - c.Assert(executor.GetStmtLabel(stmtNode), Equals, tt.label) + err = plannercore.Preprocess(tk.Session(), stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) + require.NoError(t, err) + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmtNode, preprocessorReturn.InfoSchema) + require.NoError(t, err) + require.Equal(t, tt.label, executor.GetStmtLabel(stmtNode)) } } diff --git a/executor/oomtest/oom_test.go b/executor/oomtest/oom_test.go index b6a4d452bbe4a..e58d16053cd2b 100644 --- a/executor/oomtest/oom_test.go +++ b/executor/oomtest/oom_test.go @@ -41,7 +41,7 @@ func TestMain(m *testing.M) { conf.OOMAction = config.OOMActionLog }) opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/executor/opt_rule_blacklist.go b/executor/opt_rule_blacklist.go index b9c8318d1bc30..5773f80efe7a2 100644 --- a/executor/opt_rule_blacklist.go +++ b/executor/opt_rule_blacklist.go @@ -37,11 +37,7 @@ func (e *ReloadOptRuleBlacklistExec) Next(ctx context.Context, _ *chunk.Chunk) e // LoadOptRuleBlacklist loads the latest data from table mysql.opt_rule_blacklist. func LoadOptRuleBlacklist(ctx sessionctx.Context) (err error) { exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, "select HIGH_PRIORITY name from mysql.opt_rule_blacklist") - if err != nil { - return err - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "select HIGH_PRIORITY name from mysql.opt_rule_blacklist") if err != nil { return err } diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index 809b4c5247e5a..e9bf204c1dad4 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -392,7 +392,6 @@ func (s *testSerialSuite) TestApplyWithOtherFeatures(c *C) { tk.MustExec("set tidb_enable_parallel_apply=true") // collation 1 - collate.SetNewCollationEnabledForTest(true) tk.MustExec("drop table if exists t, t1") tk.MustExec("create table t(a varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b int)") tk.MustExec("create table t1(a varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci, b int)") @@ -405,6 +404,7 @@ func (s *testSerialSuite) TestApplyWithOtherFeatures(c *C) { sql = "select (select min(t1.b) from t1 where t1.a >= t.a and t1.b >= t.b), (select sum(t1.b) from t1 where t1.a >= t.a and t1.b >= t.b) from t" tk.MustQuery(sql).Sort().Check(testkit.Rows("1 10", "2 9", "3 7", "4 4")) collate.SetNewCollationEnabledForTest(false) + defer collate.SetNewCollationEnabledForTest(true) // plan cache orgEnable := core.PreparedPlanCacheEnabled() diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 55494ea07e6cc..03c416186b505 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -18,20 +18,26 @@ import ( "fmt" "math/rand" "strings" + "testing" "time" - . "github.com/pingcap/check" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/israce" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *partitionTableSuite) TestFourReader(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFourReader(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists pt") tk.MustExec(`create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) ( partition p0 values less than (4), @@ -65,8 +71,12 @@ partition p2 values less than (10))`) tk.MustQuery("select /*+ use_index(i_c, i_id) */ * from pt where id = 4 or c < 7").Sort().Check(testkit.Rows("0 0", "2 2", "4 4", "6 6")) } -func (s *partitionTableSuite) TestPartitionIndexJoin(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionIndexJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_table_partition = 1") tk.MustExec("set @@session.tidb_enable_list_partition = 1") for i := 0; i < 3; i++ { @@ -99,9 +109,14 @@ func (s *partitionTableSuite) TestPartitionIndexJoin(c *C) { } } -func (s *partitionTableSuite) TestPartitionUnionScanIndexJoin(c *C) { +func TestPartitionUnionScanIndexJoin(t *testing.T) { // For issue https://github.com/pingcap/tidb/issues/19152 - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int)) partition by range (c_int) ( partition p0 values less than (10), partition p1 values less than maxvalue)") tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int, c_str)) partition by hash (c_int) partitions 4") @@ -114,11 +129,12 @@ func (s *partitionTableSuite) TestPartitionUnionScanIndexJoin(c *C) { tk.MustExec("commit") } -func (s *partitionTableSuite) TestPointGetwithRangeAndListPartitionTable(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPointGetwithRangeAndListPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_pointget_list_hash") tk.MustExec("use test_pointget_list_hash") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -161,35 +177,38 @@ func (s *partitionTableSuite) TestPointGetwithRangeAndListPartitionTable(c *C) { // select a from t where a={x}; // the result is {x} x := rand.Intn(100) + 1 queryRange1 := fmt.Sprintf("select a from trange1 where a=%v", x) - c.Assert(tk.HasPlan(queryRange1, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryRange1, "Point_Get")) // check if PointGet is used tk.MustQuery(queryRange1).Check(testkit.Rows(fmt.Sprintf("%v", x))) queryRange2 := fmt.Sprintf("select a from trange1 where a=%v", x) - c.Assert(tk.HasPlan(queryRange2, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryRange2, "Point_Get")) // check if PointGet is used tk.MustQuery(queryRange2).Check(testkit.Rows(fmt.Sprintf("%v", x))) y := rand.Intn(12) + 1 queryList := fmt.Sprintf("select a from tlist where a=%v", y) - c.Assert(tk.HasPlan(queryList, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryList, "Point_Get")) // check if PointGet is used tk.MustQuery(queryList).Check(testkit.Rows(fmt.Sprintf("%v", y))) } // test table dual queryRange1 := "select a from trange1 where a=200" - c.Assert(tk.HasPlan(queryRange1, "TableDual"), IsTrue) // check if TableDual is used + require.True(t, tk.HasPlan(queryRange1, "TableDual")) // check if TableDual is used tk.MustQuery(queryRange1).Check(testkit.Rows()) queryRange2 := "select a from trange2 where a=200" - c.Assert(tk.HasPlan(queryRange2, "TableDual"), IsTrue) // check if TableDual is used + require.True(t, tk.HasPlan(queryRange2, "TableDual")) // check if TableDual is used tk.MustQuery(queryRange2).Check(testkit.Rows()) queryList := "select a from tlist where a=200" - c.Assert(tk.HasPlan(queryList, "TableDual"), IsTrue) // check if TableDual is used + require.True(t, tk.HasPlan(queryList, "TableDual")) // check if TableDual is used tk.MustQuery(queryList).Check(testkit.Rows()) } -func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionReaderUnderApply(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // For issue 19458. @@ -205,7 +224,7 @@ func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) { c_double double DEFAULT NULL, c_decimal decimal(12,6) DEFAULT NULL, PRIMARY KEY (c_int,c_str,c_datetime) - ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci PARTITION BY RANGE (c_int) (PARTITION p0 VALUES LESS THAN (2) ENGINE = InnoDB, PARTITION p1 VALUES LESS THAN (4) ENGINE = InnoDB, @@ -245,8 +264,11 @@ func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) { "5 naughty swartz 9.524000")) } -func (s *partitionTableSuite) TestImproveCoverage(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestImproveCoverage(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table coverage_rr ( pk1 varchar(35) NOT NULL, @@ -261,8 +283,11 @@ PRIMARY KEY (pk1,pk2)) partition by hash(pk2) partitions 4;`) tk.MustQuery("select /*+ INL_MERGE_JOIN(dt, rr) */ * from coverage_dt dt join coverage_rr rr on (dt.pk1 = rr.pk1 and dt.pk2 = rr.pk2);").Sort().Check(testkit.Rows("ios 3 ios 3 2", "linux 5 linux 5 1")) } -func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionInfoDisable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t_info_null") tk.MustExec(`CREATE TABLE t_info_null ( @@ -287,9 +312,9 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { PARTITION p202010 VALUES LESS THAN ("2020-11-01"), PARTITION p202011 VALUES LESS THAN ("2020-12-01") )`) - is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null")) - c.Assert(err, IsNil) + require.NoError(t, err) tbInfo := tbl.Meta() // Mock for a case that the tableInfo.Partition is not nil, but tableInfo.Partition.Enable is false. @@ -305,12 +330,12 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } -func (s *partitionTableSuite) TestOrderByandLimit(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestOrderByandLimit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_orderby_limit") tk.MustExec("use test_orderby_limit") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -344,7 +369,7 @@ func (s *partitionTableSuite) TestOrderByandLimit(c *C) { y := rand.Intn(2000) + 1 queryPartition := fmt.Sprintf("select * from trange use index(idx_a) where a > %v order by a, b limit %v;", x, y) queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v order by a, b limit %v;", x, y) - c.Assert(tk.HasPlan(queryPartition, "IndexLookUp"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition, "IndexLookUp")) // check if IndexLookUp is used tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } @@ -356,7 +381,7 @@ func (s *partitionTableSuite) TestOrderByandLimit(c *C) { y := rand.Intn(2000) + 1 queryPartition := fmt.Sprintf("select * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) - c.Assert(tk.HasPlan(queryPartition, "TableReader"), IsTrue) // check if tableReader is used + require.True(t, tk.HasPlan(queryPartition, "TableReader")) // check if tableReader is used tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } @@ -368,7 +393,7 @@ func (s *partitionTableSuite) TestOrderByandLimit(c *C) { y := rand.Intn(2000) + 1 queryPartition := fmt.Sprintf("select a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) - c.Assert(tk.HasPlan(queryPartition, "IndexReader"), IsTrue) // check if indexReader is used + require.True(t, tk.HasPlan(queryPartition, "IndexReader")) // check if indexReader is used tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } @@ -379,17 +404,18 @@ func (s *partitionTableSuite) TestOrderByandLimit(c *C) { y := rand.Intn(2000) + 1 queryPartition := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > 2 or b < 5 order by a, b limit %v;", y) queryRegular := fmt.Sprintf("select * from tregular where a > 2 or b < 5 order by a, b limit %v;", y) - c.Assert(tk.HasPlan(queryPartition, "IndexMerge"), IsTrue) // check if indexMerge is used + require.True(t, tk.HasPlan(queryPartition, "IndexMerge")) // check if indexMerge is used tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } } -func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestBatchGetandPointGetwithHashPartition(t *testing.T) { + + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_batchget_pointget") tk.MustExec("use test_batchget_pointget") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -415,13 +441,13 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { x := rand.Intn(100) + 1 queryHash := fmt.Sprintf("select a from thash where a=%v", x) queryRegular := fmt.Sprintf("select a from tregular where a=%v", x) - c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryHash, "Point_Get")) // check if PointGet is used tk.MustQuery(queryHash).Check(tk.MustQuery(queryRegular).Rows()) } // test empty PointGet queryHash := "select a from thash where a=200" - c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryHash, "Point_Get")) // check if PointGet is used tk.MustQuery(queryHash).Check(testkit.Rows()) // test BatchGet @@ -436,17 +462,17 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) queryRegular := fmt.Sprintf("select a from tregular where a in (%v)", strings.Join(points, ",")) - c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + require.True(t, tk.HasPlan(queryHash, "Point_Get")) // check if PointGet is used tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } } -func (s *partitionTableSuite) TestView(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_view") tk.MustExec("use test_view") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -513,12 +539,12 @@ func (s *partitionTableSuite) TestView(c *C) { } } -func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestDirectReadingwithIndexJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_dr_join") tk.MustExec("use test_dr_join") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -626,12 +652,12 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } -func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestDynamicPruningUnderIndexJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database pruing_under_index_join") tk.MustExec("use pruing_under_index_join") @@ -686,12 +712,12 @@ func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) } -func (s *partitionTableSuite) TestIssue25527(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestIssue25527(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_issue_25527") tk.MustExec("use test_issue_25527") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -725,8 +751,12 @@ func (s *partitionTableSuite) TestIssue25527(c *C) { tk.MustQuery(`select a from t2 where a in (5)`).Check(testkit.Rows("5")) } -func (s *partitionTableSuite) TestIssue25598(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue25598(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_issue_25598") tk.MustExec("use test_issue_25598") tk.MustExec(`CREATE TABLE UK_HP16726 ( @@ -748,11 +778,12 @@ func (s *partitionTableSuite) TestIssue25598(c *C) { tk.MustExec(`explain select t1. col1, t2. col1 from UK_HP16726 as t1 inner join UK_HP16726 as t2 on t1.col1 = t2.col1 where t1.col1 > -9223372036854775808 group by t1.col1, t2.col1 having t1.col1 != 9223372036854775807`) } -func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } - tk := testkit.NewTestKitWithInit(c, s.store) +func TestBatchGetforRangeandListPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_pointget") tk.MustExec("use test_pointget") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -802,11 +833,11 @@ func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { queryRegular1 := fmt.Sprintf("select a from tregular1 where a in (%v)", strings.Join(points, ",")) queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) - c.Assert(tk.HasPlan(queryHash, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + require.True(t, tk.HasPlan(queryHash, "Batch_Point_Get")) // check if BatchGet is used tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryRange := fmt.Sprintf("select a from trange where a in (%v)", strings.Join(points, ",")) - c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + require.True(t, tk.HasPlan(queryRange, "Batch_Point_Get")) // check if BatchGet is used tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) points = make([]string, 0, 10) @@ -816,7 +847,7 @@ func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { } queryRegular2 := fmt.Sprintf("select a from tregular2 where a in (%v)", strings.Join(points, ",")) queryList := fmt.Sprintf("select a from tlist where a in (%v)", strings.Join(points, ",")) - c.Assert(tk.HasPlan(queryList, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + require.True(t, tk.HasPlan(queryList, "Batch_Point_Get")) // check if BatchGet is used tk.MustQuery(queryList).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) } @@ -846,16 +877,16 @@ func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { } queryRegular := fmt.Sprintf("select a from tregular3 where a in (%v)", strings.Join(points, ",")) queryRange := fmt.Sprintf("select a from trange3 where a in (%v)", strings.Join(points, ",")) - c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + require.True(t, tk.HasPlan(queryRange, "Batch_Point_Get")) // check if BatchGet is used tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } -func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestGlobalStatsAndSQLBinding(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_global_stats") tk.MustExec("use test_global_stats") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -894,9 +925,9 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustExec("insert into tlist values " + strings.Join(listVals, ",")) // before analyzing, the planner will choose TableScan to access the 1% of records - c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) - c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) - c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + require.True(t, tk.HasPlan("select * from thash where a<100", "TableFullScan")) + require.True(t, tk.HasPlan("select * from trange where a<100", "TableFullScan")) + require.True(t, tk.HasPlan("select * from tlist where a<1", "TableFullScan")) tk.MustExec("analyze table thash") tk.MustExec("analyze table trange") @@ -913,9 +944,9 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustExec("create session binding for select * from tlist where a<100 using select * from tlist ignore index(a) where a<100") // use TableScan again since the Index(a) is ignored - c.Assert(tk.HasPlan("select * from thash where a<100", "TableFullScan"), IsTrue) - c.Assert(tk.HasPlan("select * from trange where a<100", "TableFullScan"), IsTrue) - c.Assert(tk.HasPlan("select * from tlist where a<1", "TableFullScan"), IsTrue) + require.True(t, tk.HasPlan("select * from thash where a<100", "TableFullScan")) + require.True(t, tk.HasPlan("select * from trange where a<100", "TableFullScan")) + require.True(t, tk.HasPlan("select * from tlist where a<1", "TableFullScan")) // drop SQL bindings tk.MustExec("drop session binding for select * from thash where a<100") @@ -928,12 +959,12 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } -func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestPartitionTableWithDifferentJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_partition_joins") tk.MustExec("use test_partition_joins") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -974,78 +1005,78 @@ func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { // hash_join range partition and hash partition queryHash := fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) queryRegular := fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // group 2 // hash_join range partition and regular table queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) - c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "HashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // group 3 // merge_join range partition and hash partition queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // group 4 // merge_join range partition and regular table queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) - c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "MergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // new table instances @@ -1079,28 +1110,28 @@ func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { // Currently don't support index merge join on two partition tables. Only test warning. queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v;", x1) // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) - // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // require.True(t,tk.HasPlan(queryHash, "IndexMergeJoin")) // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) tk.MustQuery(queryHash) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.a > %v;", x1, x2) // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) - // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // require.True(t,tk.HasPlan(queryHash, "IndexMergeJoin")) // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) tk.MustQuery(queryHash) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange.b > %v;", x1, x2) // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) - // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // require.True(t,tk.HasPlan(queryHash, "IndexMergeJoin")) // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) tk.MustQuery(queryHash) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.b > %v;", x1, x2) // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) - // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // require.True(t,tk.HasPlan(queryHash, "IndexMergeJoin")) // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) tk.MustQuery(queryHash) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) @@ -1109,56 +1140,56 @@ func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { // index_merge_join range partition and regualr table queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v;", x1) queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) - c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexMergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.a > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexMergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and trange.b > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexMergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.b > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexMergeJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // group 7 // index_hash_join hash partition and hash partition queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v);", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v) and thash2.a in (%v, %v);", x1, x2, x3, x4) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a > %v and thash2.b > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // group 8 // index_hash_join hash partition and hash partition queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v);", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a > %v and tregular3.b > %v;", x1, x2) queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + require.True(t, tk.HasPlan(queryHash, "IndexHashJoin")) tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) } @@ -1180,8 +1211,12 @@ type testData4Expression struct { partitions []string } -func (s *partitionTableSuite) TestDateColWithUnequalExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestDateColWithUnequalExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_datetime_unequal_expression") tk.MustExec("create database db_datetime_unequal_expression") tk.MustExec("use db_datetime_unequal_expression") @@ -1211,8 +1246,12 @@ func (s *partitionTableSuite) TestDateColWithUnequalExpression(c *C) { } } -func (s *partitionTableSuite) TestToDaysColWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestToDaysColWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_to_days_expression") tk.MustExec("create database db_to_days_expression") tk.MustExec("use db_to_days_expression") @@ -1242,8 +1281,12 @@ func (s *partitionTableSuite) TestToDaysColWithExpression(c *C) { } } -func (s *partitionTableSuite) TestWeekdayWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestWeekdayWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_weekday_expression") tk.MustExec("create database db_weekday_expression") tk.MustExec("use db_weekday_expression") @@ -1277,8 +1320,12 @@ func (s *partitionTableSuite) TestWeekdayWithExpression(c *C) { } } -func (s *partitionTableSuite) TestFloorUnixTimestampAndIntColWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFloorUnixTimestampAndIntColWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_floor_unix_timestamp_int_expression") tk.MustExec("create database db_floor_unix_timestamp_int_expression") tk.MustExec("use db_floor_unix_timestamp_int_expression") @@ -1308,8 +1355,12 @@ func (s *partitionTableSuite) TestFloorUnixTimestampAndIntColWithExpression(c *C } } -func (s *partitionTableSuite) TestUnixTimestampAndIntColWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestUnixTimestampAndIntColWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_unix_timestamp_int_expression") tk.MustExec("create database db_unix_timestamp_int_expression") tk.MustExec("use db_unix_timestamp_int_expression") @@ -1339,8 +1390,12 @@ func (s *partitionTableSuite) TestUnixTimestampAndIntColWithExpression(c *C) { } } -func (s *partitionTableSuite) TestDatetimeColAndIntColWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestDatetimeColAndIntColWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_datetime_int_expression") tk.MustExec("create database db_datetime_int_expression") tk.MustExec("use db_datetime_int_expression") @@ -1370,8 +1425,12 @@ func (s *partitionTableSuite) TestDatetimeColAndIntColWithExpression(c *C) { } } -func (s *partitionTableSuite) TestVarcharColAndIntColWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestVarcharColAndIntColWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_varchar_int_expression") tk.MustExec("create database db_varchar_int_expression") tk.MustExec("use db_varchar_int_expression") @@ -1405,8 +1464,12 @@ func (s *partitionTableSuite) TestVarcharColAndIntColWithExpression(c *C) { } } -func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestDynamicPruneModeWithExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop database if exists db_equal_expression") tk.MustExec("create database db_equal_expression") tk.MustExec("use db_equal_expression") @@ -1693,12 +1756,12 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { } } -func (s *partitionTableSuite) TestAddDropPartitions(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestAddDropPartitions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_add_drop_partition") tk.MustExec("use test_add_drop_partition") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -1727,12 +1790,12 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } -func (s *partitionTableSuite) TestMPPQueryExplainInfo(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestMPPQueryExplainInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database tiflash_partition_test") tk.MustExec("use tiflash_partition_test") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -1741,13 +1804,13 @@ func (s *partitionTableSuite) TestMPPQueryExplainInfo(c *C) { partition p0 values less than (5), partition p1 values less than (10), partition p2 values less than (15))`) - tb := testGetTableByName(c, tk.Se, "tiflash_partition_test", "t") + tb := testkit.TestGetTableByName(t, tk.Session(), "tiflash_partition_test", "t") for _, partition := range tb.Meta().GetPartitionInfo().Definitions { - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, partition.ID, true) - c.Assert(err, IsNil) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), partition.ID, true) + require.NoError(t, err) } - err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) - c.Assert(err, IsNil) + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) tk.MustExec(`insert into t values (2), (7), (12)`) tk.MustExec("set tidb_enforce_mpp=1") tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) @@ -1757,26 +1820,30 @@ func (s *partitionTableSuite) TestMPPQueryExplainInfo(c *C) { tk.MustPartition(`select * from t where a < 5 union all select * from t where a > 10`, "p2").Sort().Check(testkit.Rows("12", "2")) } -func (s *partitionTableSuite) PartitionPruningInTransaction(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestPartitionPruningInTransaction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_pruning_transaction") defer tk.MustExec(`drop database test_pruning_transaction`) tk.MustExec("use test_pruning_transaction") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec(`create table t(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11))`) tk.MustExec(`begin`) - tk.MustPartition(`select * from t`, "all") - tk.MustPartition(`select * from t where a > 4`, "p1,p2") // partition pruning can work in transactions - tk.MustPartition(`select * from t where a > 7`, "p2") + tk.MustPartitionByList(`select * from t`, []string{"p0", "p1", "p2"}) + tk.MustPartitionByList(`select * from t where a > 3`, []string{"p1", "p2"}) // partition pruning can work in transactions + tk.MustPartitionByList(`select * from t where a > 7`, []string{"p2"}) tk.MustExec(`rollback`) } -func (s *partitionTableSuite) TestIssue25253(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue25253(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database issue25253") defer tk.MustExec("drop database issue25253") tk.MustExec("use issue25253") @@ -1806,12 +1873,12 @@ func (s *partitionTableSuite) TestIssue25253(c *C) { tk.MustQuery(`select * from t`).Check(testkit.Rows()) } -func (s *partitionTableSuite) TestDML(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestDML(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_DML") defer tk.MustExec(`drop database test_DML`) tk.MustExec("use test_DML") @@ -1867,12 +1934,12 @@ func (s *partitionTableSuite) TestDML(c *C) { } } -func (s *partitionTableSuite) TestUnion(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestUnion(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_union") defer tk.MustExec(`drop database test_union`) tk.MustExec("use test_union") @@ -1918,12 +1985,12 @@ func (s *partitionTableSuite) TestUnion(c *C) { } } -func (s *partitionTableSuite) TestSubqueries(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestSubqueries(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_subquery") defer tk.MustExec(`drop database test_subquery`) tk.MustExec("use test_subquery") @@ -1984,12 +2051,12 @@ func (s *partitionTableSuite) TestSubqueries(c *C) { } } -func (s *partitionTableSuite) TestSplitRegion(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestSplitRegion(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_split_region") tk.MustExec("use test_split_region") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2021,12 +2088,12 @@ func (s *partitionTableSuite) TestSplitRegion(c *C) { tk.MustPartition(`select * from thash where a in (1, 10001, 20001)`, "p1").Sort().Check(result) } -func (s *partitionTableSuite) TestParallelApply(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestParallelApply(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_parallel_apply") tk.MustExec("use test_parallel_apply") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2161,12 +2228,12 @@ func (s *partitionTableSuite) TestParallelApply(c *C) { } } -func (s *partitionTableSuite) TestDirectReadingWithUnionScan(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestDirectReadingWithUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_unionscan") defer tk.MustExec(`drop database test_unionscan`) tk.MustExec("use test_unionscan") @@ -2236,8 +2303,12 @@ func (s *partitionTableSuite) TestDirectReadingWithUnionScan(c *C) { tk.MustExec(`rollback`) } -func (s *partitionTableSuite) TestIssue25030(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue25030(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_issue_25030") tk.MustExec("use test_issue_25030") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2261,12 +2332,12 @@ func (s *partitionTableSuite) TestIssue25030(c *C) { Check(testkit.Rows()) // can work properly without any error or panic } -func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestUnsignedPartitionColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_unsigned_partition") tk.MustExec("use test_unsigned_partition") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2310,7 +2381,7 @@ func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { for tid, tbl := range []string{"tnormal_pk", "trange_pk", "thash_pk"} { // unsigned + TableReader scanSQL := fmt.Sprintf("select * from %v use index(primary) where %v", tbl, scanCond) - c.Assert(tk.HasPlan(scanSQL, "TableReader"), IsTrue) + require.True(t, tk.HasPlan(scanSQL, "TableReader")) r := tk.MustQuery(scanSQL).Sort() if tid == 0 { rScan = r.Rows() @@ -2330,7 +2401,7 @@ func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { // unsigned + BatchGet on PK batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) - c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + require.True(t, tk.HasPlan(batchSQL, "Batch_Point_Get")) r = tk.MustQuery(batchSQL).Sort() if tid == 0 { rBatch = r.Rows() @@ -2344,7 +2415,7 @@ func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { for tid, tbl := range []string{"tnormal_uniq", "trange_uniq", "thash_uniq"} { // unsigned + IndexReader scanSQL := fmt.Sprintf("select a from %v use index(a) where %v", tbl, scanCond) - c.Assert(tk.HasPlan(scanSQL, "IndexReader"), IsTrue) + require.True(t, tk.HasPlan(scanSQL, "IndexReader")) r := tk.MustQuery(scanSQL).Sort() if tid == 0 { rScan = r.Rows() @@ -2374,7 +2445,7 @@ func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { // unsigned + BatchGet on UniqueIndex batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) - c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + require.True(t, tk.HasPlan(batchSQL, "Batch_Point_Get")) r = tk.MustQuery(batchSQL).Sort() if tid == 0 { rBatch = r.Rows() @@ -2385,12 +2456,12 @@ func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { } } -func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestDirectReadingWithAgg(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_dr_agg") tk.MustExec("use test_dr_agg") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2442,12 +2513,12 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) y := rand.Intn(1099) @@ -2455,12 +2526,12 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a in(%v, %v, %v) group by a;", x, y, z) queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition3, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a in (%v, %v, %v) group by a;", x, y, z) queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition4, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) } @@ -2474,12 +2545,12 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) y := rand.Intn(1099) @@ -2487,12 +2558,12 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a in(%v, %v, %v) group by a;", x, y, z) queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition3, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a in (%v, %v, %v) group by a;", x, y, z) queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition4, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) } @@ -2506,12 +2577,12 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) - c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) y := rand.Intn(12) + 1 @@ -2519,18 +2590,22 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a in(%v, %v, %v) group by a;", x, y, z) queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a in(%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition3, "StreamAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a in (%v, %v, %v) group by a;", x, y, z) queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a in (%v, %v, %v) group by a;", x, y, z) - c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition4, "HashAgg")) // check if IndexLookUp is used tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) } } -func (s *partitionTableSuite) TestDynamicModeByDefault(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestDynamicModeByDefault(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_dynamic_by_default") tk.MustExec(`create table trange(a int, b int, primary key(a) clustered, index idx_b(b)) partition by range(a) ( @@ -2544,13 +2619,17 @@ func (s *partitionTableSuite) TestDynamicModeByDefault(c *C) { "explain select * from thash where a>=100", } { for _, r := range tk.MustQuery(q).Rows() { - c.Assert(strings.Contains(strings.ToLower(r[0].(string)), "partitionunion"), IsFalse) + require.NotContains(t, strings.ToLower(r[0].(string)), "partitionunion") } } } -func (s *partitionTableSuite) TestIssue24636(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue24636(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_issue_24636") tk.MustExec("use test_issue_24636") @@ -2585,12 +2664,12 @@ func (s *partitionTableSuite) TestIssue24636(c *C) { tk.MustQuery(`select c,j,l from test_partition where c='428ff6a1-bb37-42ac-9883-33d7a29961e6' and a='aaa' limit 0, 200`).Check(testkit.Rows("428ff6a1-bb37-42ac-9883-33d7a29961e6 9 0")) } -func (s *partitionTableSuite) TestIdexMerge(c *C) { - if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") - } +func TestIdexMerge(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() - tk := testkit.NewTestKitWithInit(c, s.store) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_idx_merge") tk.MustExec("use test_idx_merge") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2639,12 +2718,12 @@ func (s *partitionTableSuite) TestIdexMerge(c *C) { queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b < %v;", x1, x2) queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b < %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b > %v;", x1, x2) queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) } @@ -2655,12 +2734,12 @@ func (s *partitionTableSuite) TestIdexMerge(c *C) { queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b < %v;", x1, x2) queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregualr1) */ * from tregular1 where a > %v or b < %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b > %v;", x1, x2) queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) } @@ -2670,18 +2749,22 @@ func (s *partitionTableSuite) TestIdexMerge(c *C) { x2 := rand.Intn(12) + 1 queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b < %v;", x1, x2) queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b < %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition1, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b > %v;", x1, x2) queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b > %v;", x1, x2) - c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + require.True(t, tk.HasPlan(queryPartition2, "IndexMerge")) // check if IndexLookUp is used tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) } } -func (s *partitionTableSuite) TestIssue25309(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue25309(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("create database test_issue_25309") tk.MustExec("use test_issue_25309") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -2716,8 +2799,17 @@ func (s *partitionTableSuite) TestIssue25309(c *C) { tk.MustQuery(`select tbl_5.* from tbl_500 tbl_5 where col_24 in ( select col_62 from tbl_600 where tbl_5.col_26 < 'hSvHLdQeGBNIyOFXStV' )`).Check(testkit.Rows()) } -func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestGlobalIndexScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + restoreConfig := config.RestoreFunc() + defer restoreConfig() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableGlobalIndex = true + }) + tk.MustExec("use test") tk.MustExec("drop table if exists p") tk.MustExec(`create table p (id int, c int) partition by range (c) ( partition p0 values less than (4), @@ -2728,8 +2820,17 @@ partition p2 values less than (10))`) tk.MustQuery("select id from p use index (idx)").Check(testkit.Rows("1", "3", "5", "7")) } -func (s *globalIndexSuite) TestGlobalIndexDoubleRead(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestGlobalIndexDoubleRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + restoreConfig := config.RestoreFunc() + defer restoreConfig() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableGlobalIndex = true + }) + tk.MustExec("use test") tk.MustExec("drop table if exists p") tk.MustExec(`create table p (id int, c int) partition by range (c) ( partition p0 values less than (4), @@ -2740,8 +2841,11 @@ partition p2 values less than (10))`) tk.MustQuery("select * from p use index (idx)").Check(testkit.Rows("1 3", "3 4", "5 6", "7 9")) } -func (s *partitionTableSuite) TestIssue20028(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20028(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("set @@tidb_partition_prune_mode='static-only'") @@ -2767,8 +2871,11 @@ partition p3 values less than maxvalue)`) tk.MustExec("rollback") } -func (s *partitionTableSuite) TestSelectLockOnPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectLockOnPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists pt") tk.MustExec(`create table pt (id int primary key, k int, c int, index(k)) @@ -2777,7 +2884,7 @@ partition p0 values less than (4), partition p1 values less than (7), partition p2 values less than (11))`) - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") optimisticTableReader := func() { @@ -2787,7 +2894,7 @@ partition p2 values less than (11))`) tk.MustQuery("select id, k from pt ignore index (k) where k = 5 for update").Check(testkit.Rows("5 5")) tk2.MustExec("update pt set c = c + 1 where k = 5") _, err := tk.Exec("commit") - c.Assert(err, NotNil) // Write conflict + require.Error(t, err) // Write conflict } optimisticIndexReader := func() { @@ -2798,7 +2905,7 @@ partition p2 values less than (11))`) tk.MustQuery("select k from pt where k = 5 for update").Check(testkit.Rows("5")) tk2.MustExec("update pt set c = c + 1 where k = 5") _, err := tk.Exec("commit") - c.Assert(err, NotNil) + require.Error(t, err) } optimisticIndexLookUp := func() { @@ -2808,7 +2915,7 @@ partition p2 values less than (11))`) tk.MustQuery("select c, k from pt use index (k) where k = 5 for update").Check(testkit.Rows("5 5")) tk2.MustExec("update pt set c = c + 1 where k = 5") _, err := tk.Exec("commit") - c.Assert(err, NotNil) + require.Error(t, err) } pessimisticTableReader := func() { @@ -2826,7 +2933,7 @@ partition p2 values less than (11))`) // Check the operation in the goroutine is blocked, if not the first result in // the channel should be 1. - c.Assert(<-ch, Equals, 2) + require.Equal(t, 2, <-ch) tk.MustExec("commit") <-ch @@ -2848,7 +2955,7 @@ partition p2 values less than (11))`) ch <- 2 // Check the operation in the goroutine is blocked, - c.Assert(<-ch, Equals, 2) + require.Equal(t, 2, <-ch) tk.MustExec("commit") <-ch @@ -2869,7 +2976,7 @@ partition p2 values less than (11))`) ch <- 2 // Check the operation in the goroutine is blocked, - c.Assert(<-ch, Equals, 2) + require.Equal(t, 2, <-ch) tk.MustExec("commit") <-ch @@ -2898,8 +3005,12 @@ partition p2 values less than (11))`) } } -func (s *globalIndexSuite) TestIssue21731(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue21731(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists p, t") tk.MustExec("create table t (a int, b int, unique index idx(a)) partition by list columns(b) (partition p0 values in (1), partition p1 values in (2));") } @@ -2910,17 +3021,17 @@ type testOutput struct { Res []string } -func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { +func verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { for i, tt := range input { var isSelect = false if strings.HasPrefix(strings.ToLower(tt), "select ") { isSelect = true } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt if isSelect { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) } else { // Just verify SELECT (also avoid double INSERTs during record) output[i].Res = nil @@ -2936,8 +3047,11 @@ func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []s } } -func (s *testSuiteWithData) TestRangePartitionBoundariesEq(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesEq(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("CREATE DATABASE TestRangePartitionBoundaries") @@ -2954,12 +3068,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) TestRangePartitionBoundariesNe(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesNe(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("CREATE DATABASE TestRangePartitionBoundariesNe") @@ -2979,12 +3096,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenM(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesBetweenM(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenM") defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenM") @@ -2999,13 +3119,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenS(c *C) { - c.Skip("unstable, skip it and fix it before 20210624") - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesBetweenS(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenS") defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenS") @@ -3024,12 +3146,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) TestRangePartitionBoundariesLtM(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesLtM(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("create database TestRangePartitionBoundariesLtM") @@ -3045,12 +3170,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) TestRangePartitionBoundariesLtS(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionBoundariesLtS(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("create database TestRangePartitionBoundariesLtS") @@ -3070,12 +3198,15 @@ PARTITION BY RANGE (a) ( var input []string var output []testOutput - s.testData.GetTestCases(c, &input, &output) - s.verifyPartitionResult(tk, input, output) + executorSuiteData.GetTestCases(t, &input, &output) + verifyPartitionResult(tk, input, output) } -func (s *partitionTableSuite) TestIssue25528(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue25528(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode = 'static'") tk.MustExec("use test") tk.MustExec("create table issue25528 (id int primary key, balance DECIMAL(10, 2), balance2 DECIMAL(10, 2) GENERATED ALWAYS AS (-balance) VIRTUAL, created_at TIMESTAMP) PARTITION BY HASH(id) PARTITIONS 8") @@ -3089,15 +3220,23 @@ func (s *partitionTableSuite) TestIssue25528(c *C) { tk.MustQuery("select * from issue25528 where c1 in (3, 4) order by c2 for update;").Check(testkit.Rows("3 3 3 3", "4 4 4 4")) } -func (s *partitionTableSuite) TestIssue26251(c *C) { - tk1 := testkit.NewTestKit(c, s.store) +func TestIssue26251(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) + restoreConfig := config.RestoreFunc() + defer restoreConfig() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableGlobalIndex = true + }) tk1.MustExec("use test") tk1.MustExec("create table tp (id int primary key) partition by range (id) (partition p0 values less than (100));") tk1.MustExec("create table tn (id int primary key);") tk1.MustExec("insert into tp values(1),(2);") tk1.MustExec("insert into tn values(1),(2);") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk1.MustExec("begin pessimistic") @@ -3117,6 +3256,10 @@ func (s *partitionTableSuite) TestIssue26251(c *C) { tk1.MustExec("rollback") case <-ch: // Unexpected, test fail. - c.Fail() + t.Fail() } + + // Clean up + <-ch + tk2.MustExec("rollback") } diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 71cfec7ed9c1f..48c9678991d9b 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -17,8 +17,8 @@ package executor import ( "context" "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" @@ -26,18 +26,10 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -var _ = Suite(&pkgTestSuite{}) -var _ = SerialSuites(&pkgTestSerialSuite{}) - -type pkgTestSuite struct { -} - -type pkgTestSerialSuite struct { -} - -func (s *pkgTestSuite) TestNestedLoopApply(c *C) { +func TestNestedLoopApply(t *testing.T) { ctx := context.Background() sctx := mock.NewContext() col0 := &expression.Column{Index: 0, RetType: types.NewFieldType(mysql.TypeLong)} @@ -88,20 +80,20 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { it := chunk.NewIterator4Chunk(joinChk) for rowIdx := 1; ; { err := join.Next(ctx, joinChk) - c.Check(err, IsNil) + require.NoError(t, err) if joinChk.NumRows() == 0 { break } for row := it.Begin(); row != it.End(); row = it.Next() { correctResult := fmt.Sprintf("%v %v", rowIdx, rowIdx) obtainedResult := fmt.Sprintf("%v %v", row.GetInt64(0), row.GetInt64(1)) - c.Check(obtainedResult, Equals, correctResult) + require.Equal(t, correctResult, obtainedResult) rowIdx++ } } } -func (s *pkgTestSuite) TestMoveInfoSchemaToFront(c *C) { +func TestMoveInfoSchemaToFront(t *testing.T) { dbss := [][]string{ {}, {"A", "B", "C", "a", "b", "c"}, @@ -124,9 +116,9 @@ func (s *pkgTestSuite) TestMoveInfoSchemaToFront(c *C) { } for i, dbs := range wanted { - c.Check(len(dbss[i]), Equals, len(dbs)) + require.Equal(t, len(dbs), len(dbss[i])) for j, db := range dbs { - c.Check(dbss[i][j], Equals, db) + require.Equal(t, db, dbss[i][j]) } } } diff --git a/executor/point_get.go b/executor/point_get.go index 2377e58feddee..69c2f546583d5 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -38,6 +38,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/rowcodec" "github.com/tikv/client-go/v2/txnkv/txnsnapshot" ) @@ -309,8 +310,22 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { if len(val) == 0 { if e.idxInfo != nil && !isCommonHandleRead(e.tblInfo, e.idxInfo) && !e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", - e.idxInfo.Name.O, e.handle) + return (&consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return key + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + return e.idxKey + }, + Tbl: e.tblInfo, + Idx: e.idxInfo, + Sctx: e.ctx, + }).ReportLookupInconsistent(ctx, + 1, 0, + []kv.Handle{e.handle}, + []kv.Handle{e.handle}, + []consistency.RecordData{{}}, + ) } return nil } @@ -364,7 +379,7 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro } if e.lock { seVars := e.ctx.GetSessionVars() - lockCtx := newLockCtx(seVars, e.lockWaitTime) + lockCtx := newLockCtx(seVars, e.lockWaitTime, 1) lockCtx.InitReturnValues(1) err := doLockKeys(ctx, e.ctx, lockCtx, key) if err != nil { diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 08f9f4ce954d8..8f3bc8e7e5727 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -223,186 +223,6 @@ func (s *testPointGetSuite) TestDistinctPlan(c *C) { tk.MustQuery("select distinct b from test_distinct where id in (123456789101112131,123456789101112132);").Check(testkit.Rows("223456789101112131")) } -func (s *testPointGetSuite) TestPointGetCharPK(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(4) primary key, b char(4));`) - tk.MustExec(`insert into t values("aa", "bb");`) - - // Test CHAR type. - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t where a = "aab";`).Check(testkit.Rows()) - - tk.MustExec(`truncate table t;`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - - // Test CHAR BINARY. - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2) binary primary key, b char(2));`) - tk.MustExec(`insert into t values(" ", " ");`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "";`).Check(testkit.Rows(` `)) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - -} - -func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2) primary key, b char(2));`) - tk.MustExec(`insert into t values("aa", "bb");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) - - tk.MustExec(`truncate table t;`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - - // Test CHAR BINARY. - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2) binary primary key, b char(2));`) - tk.MustExec(`insert into t values(" ", " ");`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "";`).Check(testkit.Rows(` `)) - tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) - - // Test both wildcard and column name exist in select field list - tk.MustExec(`set @@sql_mode="";`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2) primary key, b char(2));`) - tk.MustExec(`insert into t values("aa", "bb");`) - tk.MustPointGet(`select *, a from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb aa`)) - - // Test using table alias in field list - tk.MustPointGet(`select tmp.* from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb aa bb`)) - tk.MustPointGet(`select tmp.* from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.HasPlan(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`, "Point_Get") - tk.MustQuery(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) - - // Test using table alias in where clause - tk.MustPointGet(`select * from t tmp where tmp.a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select a, b from t tmp where tmp.a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select *, a, b from t tmp where tmp.a = "aa";`).Check(testkit.Rows(`aa bb aa bb`)) - - // Unknown table name in where clause and field list - err := tk.ExecToErr(`select a from t where xxxxx.a = "aa"`) - c.Assert(err, ErrorMatches, ".*Unknown column 'xxxxx.a' in 'where clause'") - err = tk.ExecToErr(`select xxxxx.a from t where a = "aa"`) - c.Assert(err, ErrorMatches, ".*Unknown column 'xxxxx.a' in 'field list'") - - // When an alias is provided, it completely hides the actual name of the table. - err = tk.ExecToErr(`select a from t tmp where t.a = "aa"`) - c.Assert(err, ErrorMatches, ".*Unknown column 't.a' in 'where clause'") - err = tk.ExecToErr(`select t.a from t tmp where a = "aa"`) - c.Assert(err, ErrorMatches, ".*Unknown column 't.a' in 'field list'") - err = tk.ExecToErr(`select t.* from t tmp where a = "aa"`) - c.Assert(err, ErrorMatches, ".*Unknown table 't'") -} - -func (s *testPointGetSuite) TestIndexLookupChar(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2), b char(2), index idx_1(a));`) - tk.MustExec(`insert into t values("aa", "bb");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustIndexLookup(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustIndexLookup(`select * from t where a = "aab";`).Check(testkit.Rows()) - - // Test query with table alias - tk.MustIndexLookup(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustIndexLookup(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) - - tk.MustExec(`truncate table t;`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - - // Test CHAR BINARY. - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a char(2) binary, b char(2), index idx_1(a));`) - tk.MustExec(`insert into t values(" ", " ");`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = "";`).Check(testkit.Rows(` `)) - tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) - -} - -func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec(`use test;`) - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a varchar(2) primary key, b varchar(2));`) - tk.MustExec(`insert into t values("aa", "bb");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t where a = "aab";`).Check(testkit.Rows()) - - tk.MustExec(`truncate table t;`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - - // // Test VARCHAR BINARY. - tk.MustExec(`drop table if exists t;`) - tk.MustExec(`create table t(a varchar(2) binary primary key, b varchar(2));`) - tk.MustExec(`insert into t values(" ", " ");`) - tk.MustExec(`insert into t values("a ", "b ");`) - - tk.MustExec(`set @@sql_mode="";`) - tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows(` `)) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - -} - func (s *testPointGetSuite) TestPointGetBinaryPK(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test;`) diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 3c0fabef5b5cd..f0e61ae05cee9 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -22,7 +22,6 @@ import ( "sync/atomic" "testing" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -150,6 +149,40 @@ func TestPreparedStmtWithHint(t *testing.T) { require.Equal(t, int32(1), atomic.LoadInt32(&sm.killed)) } +func TestPreparedNullParam(t *testing.T) { + store, dom, err := newStoreWithBootstrap() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + dom.Close() + }() + orgEnable := plannercore.PreparedPlanCacheEnabled() + defer func() { + plannercore.SetPreparedPlanCache(orgEnable) + }() + + flags := []bool{false, true} + for _, flag := range flags { + plannercore.SetPreparedPlanCache(flag) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@tidb_enable_collect_execution_info=0") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int not null, KEY id (id))") + tk.MustExec("insert into t values (1), (2), (3)") + + tk.MustExec("prepare stmt from 'select * from t where id = ?'") + tk.MustExec("set @a= null") + tk.MustQuery("execute stmt using @a").Check(testkit.Rows()) + + tkProcess := tk.Session().ShowProcess() + ps := []*util.ProcessInfo{tkProcess} + tk.Session().SetSessionManager(&mockSessionManager1{PS: ps}) + tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( + "TableDual_5 8000.00 root rows:0")) + } +} + func TestIssue29850(t *testing.T) { store, dom, err := newStoreWithBootstrap() require.NoError(t, err) @@ -1325,7 +1358,7 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { if israce.RaceEnabled { t.Skip("race test for this case takes too long time") } - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { @@ -1338,7 +1371,6 @@ func TestPrepareStmtAfterIsolationReadChange(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") // create virtual tiflash replica. - dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) require.True(t, exists) diff --git a/executor/reload_expr_pushdown_blacklist.go b/executor/reload_expr_pushdown_blacklist.go index ea873080727b3..1511e7a280195 100644 --- a/executor/reload_expr_pushdown_blacklist.go +++ b/executor/reload_expr_pushdown_blacklist.go @@ -39,11 +39,7 @@ func (e *ReloadExprPushdownBlacklistExec) Next(ctx context.Context, _ *chunk.Chu // LoadExprPushdownBlacklist loads the latest data from table mysql.expr_pushdown_blacklist. func LoadExprPushdownBlacklist(ctx sessionctx.Context) (err error) { exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, "select HIGH_PRIORITY name, store_type from mysql.expr_pushdown_blacklist") - if err != nil { - return err - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "select HIGH_PRIORITY name, store_type from mysql.expr_pushdown_blacklist") if err != nil { return err } diff --git a/executor/revoke_test.go b/executor/revoke_test.go index 185ba00129fdd..9607736a3ee65 100644 --- a/executor/revoke_test.go +++ b/executor/revoke_test.go @@ -17,19 +17,22 @@ package executor_test import ( "fmt" "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestRevokeGlobal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRevokeGlobal(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) _, err := tk.Exec(`REVOKE ALL PRIVILEGES ON *.* FROM 'nonexistuser'@'host'`) - c.Assert(err, NotNil) + require.Error(t, err) // Create a new user. createUserSQL := `CREATE USER 'testGlobalRevoke'@'localhost' IDENTIFIED BY '123';` @@ -53,14 +56,16 @@ func (s *testSuite1) TestRevokeGlobal(c *C) { } } -func (s *testSuite1) TestRevokeDBScope(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRevokeDBScope(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // Create a new user. tk.MustExec(`CREATE USER 'testDBRevoke'@'localhost' IDENTIFIED BY '123';`) tk.MustExec(`GRANT ALL ON test.* TO 'testDBRevoke'@'localhost';`) _, err := tk.Exec(`REVOKE ALL PRIVILEGES ON nonexistdb.* FROM 'testDBRevoke'@'localhost'`) - c.Assert(err, NotNil) + require.Error(t, err) // Revoke each priv from the user. for _, v := range mysql.AllDBPrivs { @@ -73,15 +78,17 @@ func (s *testSuite1) TestRevokeDBScope(c *C) { } } -func (s *testSuite1) TestRevokeTableScope(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRevokeTableScope(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // Create a new user. tk.MustExec(`CREATE USER 'testTblRevoke'@'localhost' IDENTIFIED BY '123';`) tk.MustExec(`CREATE TABLE test.test1(c1 int);`) tk.MustExec(`GRANT ALL PRIVILEGES ON test.test1 TO 'testTblRevoke'@'localhost';`) _, err := tk.Exec(`REVOKE ALL PRIVILEGES ON test.nonexisttable FROM 'testTblRevoke'@'localhost'`) - c.Assert(err, NotNil) + require.Error(t, err) // Make sure all the table privs for new user is Y. res := tk.MustQuery(`SELECT Table_priv FROM mysql.tables_priv WHERE User="testTblRevoke" and host="localhost" and db="test" and Table_name="test1"`) @@ -92,9 +99,9 @@ func (s *testSuite1) TestRevokeTableScope(c *C) { sql := fmt.Sprintf("REVOKE %s ON test.test1 FROM 'testTblRevoke'@'localhost';", mysql.Priv2Str[v]) tk.MustExec(sql) rows := tk.MustQuery(`SELECT Table_priv FROM mysql.tables_priv WHERE User="testTblRevoke" and host="localhost" and db="test" and Table_name="test1";`).Rows() - c.Assert(rows, HasLen, 1) + require.Len(t, rows, 1) row := rows[0] - c.Assert(row, HasLen, 1) + require.Len(t, row, 1) op := v.SetString() found := false @@ -104,7 +111,7 @@ func (s *testSuite1) TestRevokeTableScope(c *C) { break } } - c.Assert(found, IsFalse, Commentf("%s", mysql.Priv2SetStr[v])) + require.False(t, found, "%s", mysql.Priv2SetStr[v]) } // Revoke all table scope privs. @@ -112,8 +119,10 @@ func (s *testSuite1) TestRevokeTableScope(c *C) { tk.MustQuery(`SELECT Table_priv FROM mysql.Tables_priv WHERE User="testTblRevoke" and host="localhost" and db="test" and Table_name="test1"`).Check(testkit.Rows("")) } -func (s *testSuite1) TestRevokeColumnScope(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRevokeColumnScope(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // Create a new user. tk.MustExec(`CREATE USER 'testColRevoke'@'localhost' IDENTIFIED BY '123';`) tk.MustExec(`CREATE TABLE test.test3(c1 int, c2 int);`) @@ -127,11 +136,11 @@ func (s *testSuite1) TestRevokeColumnScope(c *C) { tk.MustExec(grantSQL) rows := tk.MustQuery(checkSQL).Rows() - c.Assert(rows, HasLen, 1) + require.Len(t, rows, 1) row := rows[0] - c.Assert(row, HasLen, 1) + require.Len(t, row, 1) p := fmt.Sprintf("%v", row[0]) - c.Assert(strings.Index(p, mysql.Priv2SetStr[v]), Greater, -1) + require.Greater(t, strings.Index(p, mysql.Priv2SetStr[v]), -1) tk.MustExec(revokeSQL) tk.MustQuery(checkSQL).Check(testkit.Rows("")) @@ -145,18 +154,20 @@ func (s *testSuite1) TestRevokeColumnScope(c *C) { // Make sure all the column privs for granted user are in the Column_priv set. for _, v := range mysql.AllColumnPrivs { rows := tk.MustQuery(`SELECT Column_priv FROM mysql.Columns_priv WHERE User="testCol1Revoke" and host="localhost" and db="test" and Table_name="test3" and Column_name="c2";`).Rows() - c.Assert(rows, HasLen, 1) + require.Len(t, rows, 1) row := rows[0] - c.Assert(row, HasLen, 1) + require.Len(t, row, 1) p := fmt.Sprintf("%v", row[0]) - c.Assert(strings.Index(p, mysql.Priv2SetStr[v]), Greater, -1) + require.Greater(t, strings.Index(p, mysql.Priv2SetStr[v]), -1) } tk.MustExec("REVOKE ALL(c2) ON test3 FROM 'testCol1Revoke'@'localhost'") tk.MustQuery(`SELECT Column_priv FROM mysql.Columns_priv WHERE User="testCol1Revoke" and host="localhost" and db="test" and Table_name="test3"`).Check(testkit.Rows("")) } -func (s *testSuite1) TestRevokeDynamicPrivs(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRevokeDynamicPrivs(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("DROP USER if exists dyn") tk.MustExec("create user dyn") @@ -166,7 +177,7 @@ func (s *testSuite1) TestRevokeDynamicPrivs(c *C) { // try revoking only on test.* - should fail: _, err := tk.Exec("REVOKE BACKUP_Admin,system_variables_admin ON test.* FROM dyn") - c.Assert(terror.ErrorEqual(err, executor.ErrIllegalPrivilegeLevel), IsTrue) + require.True(t, terror.ErrorEqual(err, executor.ErrIllegalPrivilegeLevel)) // privs should still be intact: tk.MustQuery("SELECT * FROM mysql.global_grants WHERE `Host` = '%' AND `User` = 'dyn' ORDER BY user,host,priv,with_grant_option").Check(testkit.Rows("dyn % BACKUP_ADMIN N")) @@ -195,9 +206,11 @@ func (s *testSuite1) TestRevokeDynamicPrivs(c *C) { tk.MustQuery("SELECT * FROM mysql.global_grants WHERE `Host` = '%' AND `User` = 'dyn' ORDER BY user,host,priv,with_grant_option").Check(testkit.Rows("dyn % SYSTEM_VARIABLES_ADMIN Y")) } -func (s *testSuite1) TestRevokeOnNonExistTable(c *C) { +func TestRevokeOnNonExistTable(t *testing.T) { // issue #28533 - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DATABASE d1;") defer tk.MustExec("DROP DATABASE IF EXISTS d1;") diff --git a/executor/rowid_test.go b/executor/rowid_test.go index d7664f1e0fb1c..5d49062d874e0 100644 --- a/executor/rowid_test.go +++ b/executor/rowid_test.go @@ -15,18 +15,23 @@ package executor_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestExportRowID(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.GetSessionVars().AllowWriteRowID = true +func TestExportRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.Session().GetSessionVars().AllowWriteRowID = true defer func() { - tk.Se.GetSessionVars().AllowWriteRowID = false + tk.Session().GetSessionVars().AllowWriteRowID = false }() + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int)") tk.MustExec("insert t values (1, 7), (1, 8), (1, 9)") @@ -48,25 +53,27 @@ func (s *testSuite1) TestExportRowID(c *C) { tk.MustExec("create table s (a int primary key)") tk.MustExec("insert s values (1)") _, err := tk.Exec("insert s (a, _tidb_rowid) values (1, 2)") - c.Assert(err, NotNil) + require.Error(t, err) err = tk.ExecToErr("select _tidb_rowid from s") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("update s set a = 2 where _tidb_rowid = 1") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("delete from s where _tidb_rowid = 1") - c.Assert(err, NotNil) + require.Error(t, err) // Make sure "AllowWriteRowID" is a session variable. - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") _, err = tk1.Exec("insert into t (a, _tidb_rowid) values(10, 1);") - c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + require.EqualError(t, err, "insert, update and replace statements for _tidb_rowid are not supported.") } -func (s *testSuite1) TestNotAllowWriteRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNotAllowWriteRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table tt(id binary(10), c int, primary key(id));") tk.MustExec("insert tt values (1, 10);") // select statement @@ -74,13 +81,13 @@ func (s *testSuite1) TestNotAllowWriteRowID(c *C) { Check(testkit.Rows("1\x00\x00\x00\x00\x00\x00\x00\x00\x00 10 1")) // insert statement _, err := tk.Exec("insert into tt (id, c, _tidb_rowid) values(30000,10,1);") - c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + require.EqualError(t, err, "insert, update and replace statements for _tidb_rowid are not supported.") // replace statement _, err = tk.Exec("replace into tt (id, c, _tidb_rowid) values(30000,10,1);") - c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + require.EqualError(t, err, "insert, update and replace statements for _tidb_rowid are not supported.") // update statement _, err = tk.Exec("update tt set id = 2, _tidb_rowid = 1 where _tidb_rowid = 1") - c.Assert(err.Error(), Equals, "insert, update and replace statements for _tidb_rowid are not supported.") + require.EqualError(t, err, "insert, update and replace statements for _tidb_rowid are not supported.") tk.MustExec("update tt set id = 2 where _tidb_rowid = 1") tk.MustExec("admin check table tt;") tk.MustExec("drop table tt") @@ -91,8 +98,10 @@ func (s *testSuite1) TestNotAllowWriteRowID(c *C) { } // Test for https://github.com/pingcap/tidb/issues/22029. -func (s *testSuite3) TestExplicitInsertRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplicitInsertRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_explicit_insert_rowid;") tk.MustExec("create database test_explicit_insert_rowid;") tk.MustExec("use test_explicit_insert_rowid;") diff --git a/executor/sample.go b/executor/sample.go index 5d57c9a792443..10d2b2f3d1716 100644 --- a/executor/sample.go +++ b/executor/sample.go @@ -17,7 +17,6 @@ package executor import ( "context" "sort" - "time" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -154,14 +153,14 @@ func (s *tableRegionSampler) pickRanges(count int) ([]kv.KeyRange, error) { } func (s *tableRegionSampler) writeChunkFromRanges(ranges []kv.KeyRange, req *chunk.Chunk) error { - decLoc, sysLoc := s.ctx.GetSessionVars().Location(), time.UTC + decLoc := s.ctx.GetSessionVars().Location() cols, decColMap, err := s.buildSampleColAndDecodeColMap() if err != nil { return err } rowDecoder := decoder.NewRowDecoder(s.table, cols, decColMap) err = s.scanFirstKVForEachRange(ranges, func(handle kv.Handle, value []byte) error { - _, err := rowDecoder.DecodeAndEvalRowWithMap(s.ctx, handle, value, decLoc, sysLoc, s.rowMap) + _, err := rowDecoder.DecodeAndEvalRowWithMap(s.ctx, handle, value, decLoc, s.rowMap) if err != nil { return err } diff --git a/executor/sample_test.go b/executor/sample_test.go index d200919081e84..4a1ec83c818e0 100644 --- a/executor/sample_test.go +++ b/executor/sample_test.go @@ -15,59 +15,21 @@ package executor_test import ( - "flag" "fmt" "sync/atomic" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" - "github.com/tikv/client-go/v2/testutils" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testTableSampleSuite{}) - -type testTableSampleSuite struct { - cluster testutils.Cluster - store kv.Storage - domain *domain.Domain -} - -func (s *testTableSampleSuite) SetUpSuite(c *C) { - flag.Lookup("mockTikv") - useMockTikv := *mockTikv - if useMockTikv { - store, err := mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockstore.BootstrapWithSingleStore(c) - s.cluster = c - }), - ) - c.Assert(err, IsNil) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() - } - d, err := session.BootstrapSession(s.store) - c.Assert(err, IsNil) - d.SetStatsUpdating(true) - s.domain = d -} - -func (s *testTableSampleSuite) TearDownSuite(c *C) { - s.domain.Close() - c.Assert(s.store.Close(), IsNil) -} - -func (s *testTableSampleSuite) initSampleTest(c *C) *testkit.TestKit { +func createSampleTestkit(t *testing.T, store kv.Storage) *testkit.TestKit { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_table_sample;") tk.MustExec("create database test_table_sample;") tk.MustExec("use test_table_sample;") @@ -75,10 +37,12 @@ func (s *testTableSampleSuite) initSampleTest(c *C) *testkit.TestKit { return tk } -func (s *testTableSampleSuite) TestTableSampleBasic(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleBasic(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int);") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustQuery("select * from t tablesample regions();").Check(testkit.Rows()) tk.MustExec("insert into t values (0), (1000), (2000);") @@ -88,7 +52,7 @@ func (s *testTableSampleSuite) TestTableSampleBasic(c *C) { tk.MustExec("alter table t add column c int as (a + 1);") tk.MustQuery("select c from t tablesample regions();").Check(testkit.Rows("1")) tk.MustQuery("select c, _tidb_rowid from t tablesample regions();").Check(testkit.Rows("1 1")) - c.Assert(tk.HasPlan("select * from t tablesample regions();", "TableSample"), IsTrue) + require.True(t, tk.HasPlan("select * from t tablesample regions();", "TableSample")) tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a BIGINT PRIMARY KEY AUTO_RANDOM(3), b int auto_increment, key(b)) pre_split_regions=8;") @@ -105,14 +69,16 @@ func (s *testTableSampleSuite) TestTableSampleBasic(c *C) { tk.MustQuery("select a from t tablesample regions() limit 2;").Check(testkit.Rows("a", "b")) } -func (s *testTableSampleSuite) TestTableSampleMultiRegions(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleMultiRegions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int) shard_row_id_bits = 2 pre_split_regions = 2;") for i := 0; i < 100; i++ { tk.MustExec("insert into t values (?);", i) } rows := tk.MustQuery("select * from t tablesample regions();").Rows() - c.Assert(len(rows), Equals, 4) + require.Len(t, rows, 4) tk.MustQuery("select a from t tablesample regions() order by a limit 1;").Check(testkit.Rows("0")) tk.MustQuery("select a from t tablesample regions() where a = 0;").Check(testkit.Rows("0")) @@ -121,13 +87,15 @@ func (s *testTableSampleSuite) TestTableSampleMultiRegions(c *C) { tk.MustExec("insert into t2 values (?);", i) } rows = tk.MustQuery("select * from t tablesample regions(), t2 tablesample regions();").Rows() - c.Assert(len(rows), Equals, 16) + require.Len(t, rows, 16) tk.MustQuery("select count(*) from t tablesample regions();").Check(testkit.Rows("4")) tk.MustExec("drop table t2;") } -func (s *testTableSampleSuite) TestTableSampleNoSplitTable(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleNoSplitTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) tk.MustExec("drop table if exists t1;") tk.MustExec("drop table if exists t2;") @@ -136,25 +104,29 @@ func (s *testTableSampleSuite) TestTableSampleNoSplitTable(c *C) { tk.MustExec("insert into t2 values(1);") rows := tk.MustQuery("select * from t1 tablesample regions();").Rows() rows2 := tk.MustQuery("select * from t2 tablesample regions();").Rows() - c.Assert(len(rows), Equals, 0) - c.Assert(len(rows2), Equals, 1) + require.Len(t, rows, 0) + require.Len(t, rows2, 1) } -func (s *testTableSampleSuite) TestTableSamplePlan(c *C) { - tk := s.initSampleTest(c) +func TestTableSamplePlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a bigint, b int default 10);") tk.MustExec("split table t between (0) and (100000) regions 4;") tk.MustExec("insert into t(a) values (1), (2), (3);") rows := tk.MustQuery("explain analyze select a from t tablesample regions();").Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) tableSample := fmt.Sprintf("%v", rows[1]) - c.Assert(tableSample, Matches, ".*TableSample.*") + require.Regexp(t, ".*TableSample.*", tableSample) } -func (s *testTableSampleSuite) TestTableSampleSchema(c *C) { - tk := s.initSampleTest(c) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn +func TestTableSampleSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn // Clustered index tk.MustExec("create table t (a varchar(255) primary key, b bigint);") tk.MustExec("insert into t values ('b', 100), ('y', 100);") @@ -182,8 +154,10 @@ func (s *testTableSampleSuite) TestTableSampleSchema(c *C) { tk.MustQuery("select a from t tablesample regions();").Check(testkit.Rows("1")) } -func (s *testTableSampleSuite) TestTableSampleInvalid(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleInvalid(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int, b varchar(255));") tk.MustExec("insert into t values (1, 'abc');") tk.MustExec("create view v as select * from t;") @@ -196,8 +170,10 @@ func (s *testTableSampleSuite) TestTableSampleInvalid(c *C) { tk.MustGetErrCode("select a from t tablesample ();", errno.ErrInvalidTableSample) } -func (s *testTableSampleSuite) TestTableSampleWithTiDBRowID(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleWithTiDBRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int, b varchar(255));") tk.MustExec("insert into t values (1, 'abc');") tk.MustQuery("select _tidb_rowid from t tablesample regions();").Check(testkit.Rows("1")) @@ -206,19 +182,21 @@ func (s *testTableSampleSuite) TestTableSampleWithTiDBRowID(c *C) { tk.MustQuery("select b, _tidb_rowid, a from t tablesample regions();").Check(testkit.Rows("abc 1 1")) } -func (s *testTableSampleSuite) TestTableSampleWithPartition(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleWithPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int, b varchar(255), primary key (a)) partition by hash(a) partitions 2;") tk.MustExec("insert into t values (1, '1'), (2, '2'), (3, '3');") rows := tk.MustQuery("select * from t tablesample regions();").Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) tk.MustExec("delete from t;") tk.MustExec("insert into t values (1, '1');") rows = tk.MustQuery("select * from t partition (p0) tablesample regions();").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) rows = tk.MustQuery("select * from t partition (p1) tablesample regions();").Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) // Test https://github.com/pingcap/tidb/issues/27349. tk.MustExec("drop table if exists t;") @@ -232,8 +210,10 @@ func (s *testTableSampleSuite) TestTableSampleWithPartition(c *C) { Check(testkit.Rows("1", "2", "3")) // The order of _tidb_rowid should be correct. } -func (s *testTableSampleSuite) TestTableSampleGeneratedColumns(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleGeneratedColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int primary key, b int as (a + 1), c int as (b + 1), d int as (c + 1));") tk.MustQuery("split table t between (0) and (10000) regions 4;").Check(testkit.Rows("3 1")) tk.MustExec("insert into t(a) values (1), (2), (2999), (4999), (9999);") @@ -247,8 +227,10 @@ func (s *testTableSampleSuite) TestTableSampleGeneratedColumns(c *C) { testkit.Rows("1 4", "2999 3002", "9999 10002")) } -func (s *testTableSampleSuite) TestTableSampleUnionScanIgnorePendingKV(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleUnionScanIgnorePendingKV(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int primary key);") tk.MustQuery("split table t between (0) and (40000) regions 4;").Check(testkit.Rows("3 1")) tk.MustExec("insert into t values (1), (1000), (10002);") @@ -265,9 +247,12 @@ func (s *testTableSampleSuite) TestTableSampleUnionScanIgnorePendingKV(c *C) { tk.MustQuery("select * from t tablesample regions();").Check(testkit.Rows("1000", "10002", "20006", "50000")) } -func (s *testTableSampleSuite) TestTableSampleTransactionConsistency(c *C) { - tk := s.initSampleTest(c) - tk2 := s.initSampleTest(c) +func TestTableSampleTransactionConsistency(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) + tk2 := createSampleTestkit(t, store) + tk.MustExec("create table t (a int primary key);") tk.MustQuery("split table t between (0) and (40000) regions 4;").Check(testkit.Rows("3 1")) tk.MustExec("insert into t values (1), (1000), (10002);") @@ -280,8 +265,10 @@ func (s *testTableSampleSuite) TestTableSampleTransactionConsistency(c *C) { tk.MustQuery("select * from t tablesample regions();").Check(testkit.Rows("1", "10002", "20006", "50000")) } -func (s *testTableSampleSuite) TestTableSampleNotSupportedPlanWarning(c *C) { - tk := s.initSampleTest(c) +func TestTableSampleNotSupportedPlanWarning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int primary key, b int, c varchar(255));") tk.MustQuery("split table t between (0) and (10000) regions 5;").Check(testkit.Rows("4 1")) tk.MustExec("insert into t values (1000, 1, '1'), (1001, 1, '1'), (2100, 2, '2'), (4500, 3, '3');") @@ -294,13 +281,15 @@ func (s *testTableSampleSuite) TestTableSampleNotSupportedPlanWarning(c *C) { tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 8128 Invalid TABLESAMPLE: plan not supported")) } -func (s *testTableSampleSuite) TestMaxChunkSize(c *C) { - tk := s.initSampleTest(c) +func TestMaxChunkSize(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := createSampleTestkit(t, store) tk.MustExec("create table t (a int) shard_row_id_bits = 2 pre_split_regions = 2;") for i := 0; i < 100; i++ { tk.MustExec("insert into t values (?);", i) } - tk.Se.GetSessionVars().MaxChunkSize = 1 + tk.Session().GetSessionVars().MaxChunkSize = 1 rows := tk.MustQuery("select * from t tablesample regions();").Rows() - c.Assert(len(rows), Equals, 4) + require.Len(t, rows, 4) } diff --git a/executor/select_into_test.go b/executor/select_into_test.go index fc7f6d98869bc..616dcd30c45d6 100644 --- a/executor/select_into_test.go +++ b/executor/select_into_test.go @@ -19,57 +19,61 @@ import ( "os" "path/filepath" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -func cmpAndRm(expected, outfile string, c *C) { +func cmpAndRm(expected, outfile string, t *testing.T) { content, err := os.ReadFile(outfile) - c.Assert(err, IsNil) - c.Assert(string(content), Equals, expected) - c.Assert(os.Remove(outfile), IsNil) + require.NoError(t, err) + require.Equal(t, expected, string(content)) + require.NoError(t, os.Remove(outfile)) } func randomSelectFilePath(testName string) string { return filepath.Join(os.TempDir(), fmt.Sprintf("select-into-%v-%v.data", testName, time.Now().Nanosecond())) } -func (s *testSuite1) TestSelectIntoFileExists(c *C) { +func TestSelectIntoFileExists(t *testing.T) { outfile := randomSelectFilePath("TestSelectIntoFileExists") defer func() { - c.Assert(os.Remove(outfile), IsNil) + require.NoError(t, os.Remove(outfile)) }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) sql := fmt.Sprintf("select 1 into outfile %q", outfile) tk.MustExec(sql) err := tk.ExecToErr(sql) - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "already exists") || - strings.Contains(err.Error(), "file exists"), IsTrue, Commentf("err: %v", err)) - c.Assert(strings.Contains(err.Error(), outfile), IsTrue) + require.Error(t, err) + require.Truef(t, strings.Contains(err.Error(), "already exists") || strings.Contains(err.Error(), "file exists"), "err: %v", err) + require.True(t, strings.Contains(err.Error(), outfile)) } -func (s *testSuite1) TestSelectIntoOutfileTypes(c *C) { +func TestSelectIntoOutfileTypes(t *testing.T) { outfile := randomSelectFilePath("TestSelectIntoOutfileTypes") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` ( `a` bit(10) DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") tk.MustExec("INSERT INTO `t` VALUES (_binary '\\0'), (_binary '\\1'), (_binary '\\2'), (_binary '\\3');") tk.MustExec(fmt.Sprintf("SELECT * FROM t INTO OUTFILE %q", outfile)) - cmpAndRm("\x00\x00\n\x001\n\x002\n\x003\n", outfile, c) + cmpAndRm("\x00\x00\n\x001\n\x002\n\x003\n", outfile, t) tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (col ENUM ('value1','value2','value3'));") tk.MustExec("INSERT INTO t values ('value1'), ('value2');") tk.MustExec(fmt.Sprintf("SELECT * FROM t INTO OUTFILE %q", outfile)) - cmpAndRm("value1\nvalue2\n", outfile, c) + cmpAndRm("value1\nvalue2\n", outfile, t) tk.MustExec("drop table if exists t") tk.MustExec("create table t ( v json);") @@ -77,7 +81,7 @@ func (s *testSuite1) TestSelectIntoOutfileTypes(c *C) { tk.MustExec(fmt.Sprintf("SELECT * FROM t INTO OUTFILE %q", outfile)) cmpAndRm(`{"id": 1, "name": "aaa"} {"id": 2, "name": "xxx"} -`, outfile, c) +`, outfile, t) tk.MustExec("drop table if exists t") tk.MustExec("create table t (v tinyint unsigned)") @@ -85,7 +89,7 @@ func (s *testSuite1) TestSelectIntoOutfileTypes(c *C) { tk.MustExec(fmt.Sprintf("SELECT * FROM t INTO OUTFILE %q", outfile)) cmpAndRm(`0 1 -`, outfile, c) +`, outfile, t) tk.MustExec("drop table if exists t") tk.MustExec("create table t (id float(16,2)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") @@ -95,12 +99,14 @@ func (s *testSuite1) TestSelectIntoOutfileTypes(c *C) { 2.00 3.40 10.10 -`, outfile, c) +`, outfile, t) } -func (s *testSuite1) TestSelectIntoOutfileFromTable(c *C) { +func TestSelectIntoOutfileFromTable(t *testing.T) { outfile := randomSelectFilePath("TestSelectIntoOutfileFromTable") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -115,84 +121,88 @@ func (s *testSuite1) TestSelectIntoOutfileFromTable(c *C) { 2 2.2 0.20000 b 2000-02-02 00:00:00 2002-02-02 00:00:00 02:02:02 [1, 2] \N \N \N \N 2000-03-03 00:00:00 2003-03-03 00:00:00 03:03:03 [1, 2, 3] 4 4.4 0.40000 d \N \N \N \N -`, outfile, c) - c.Assert(tk.Se.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(4)) +`, outfile, t) + require.Equal(t, uint64(4), tk.Session().GetSessionVars().StmtCtx.AffectedRows()) tk.MustExec(fmt.Sprintf("select * from t into outfile %q fields terminated by ',' enclosed by '\"' escaped by '#'", outfile)) cmpAndRm(`"1","1.1","0.10000","a","2000-01-01 00:00:00","2001-01-01 00:00:00","01:01:01","[1]" "2","2.2","0.20000","b","2000-02-02 00:00:00","2002-02-02 00:00:00","02:02:02","[1, 2]" #N,#N,#N,#N,"2000-03-03 00:00:00","2003-03-03 00:00:00","03:03:03","[1, 2, 3]" "4","4.4","0.40000","d",#N,#N,#N,#N -`, outfile, c) - c.Assert(tk.Se.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(4)) +`, outfile, t) + require.Equal(t, uint64(4), tk.Session().GetSessionVars().StmtCtx.AffectedRows()) tk.MustExec(fmt.Sprintf("select * from t into outfile %q fields terminated by ',' optionally enclosed by '\"' escaped by '#'", outfile)) cmpAndRm(`1,1.1,0.10000,"a","2000-01-01 00:00:00","2001-01-01 00:00:00","01:01:01","[1]" 2,2.2,0.20000,"b","2000-02-02 00:00:00","2002-02-02 00:00:00","02:02:02","[1, 2]" #N,#N,#N,#N,"2000-03-03 00:00:00","2003-03-03 00:00:00","03:03:03","[1, 2, 3]" 4,4.4,0.40000,"d",#N,#N,#N,#N -`, outfile, c) - c.Assert(tk.Se.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(4)) +`, outfile, t) + require.Equal(t, uint64(4), tk.Session().GetSessionVars().StmtCtx.AffectedRows()) tk.MustExec(fmt.Sprintf("select * from t into outfile %q fields terminated by ',' optionally enclosed by '\"' escaped by '#' lines terminated by '<<<\n'", outfile)) cmpAndRm(`1,1.1,0.10000,"a","2000-01-01 00:00:00","2001-01-01 00:00:00","01:01:01","[1]"<<< 2,2.2,0.20000,"b","2000-02-02 00:00:00","2002-02-02 00:00:00","02:02:02","[1, 2]"<<< #N,#N,#N,#N,"2000-03-03 00:00:00","2003-03-03 00:00:00","03:03:03","[1, 2, 3]"<<< 4,4.4,0.40000,"d",#N,#N,#N,#N<<< -`, outfile, c) - c.Assert(tk.Se.GetSessionVars().StmtCtx.AffectedRows(), Equals, uint64(4)) +`, outfile, t) + require.Equal(t, uint64(4), tk.Session().GetSessionVars().StmtCtx.AffectedRows()) } -func (s *testSuite1) TestSelectIntoOutfileConstant(c *C) { +func TestSelectIntoOutfileConstant(t *testing.T) { outfile := randomSelectFilePath("TestSelectIntoOutfileConstant") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // On windows the outfile name looks like "C:\Users\genius\AppData\Local\Temp\select-into-outfile.data", // fmt.Sprintf("%q") is used otherwise the string become // "C:UsersgeniusAppDataLocalTempselect-into-outfile.data". tk.MustExec(fmt.Sprintf("select 1, 2, 3, '4', '5', '6', 7.7, 8.8, 9.9, null into outfile %q", outfile)) // test constants cmpAndRm(`1 2 3 4 5 6 7.7 8.8 9.9 \N -`, outfile, c) +`, outfile, t) tk.MustExec(fmt.Sprintf("select 1e10, 1e20, 1.234567e8, 0.000123e3, 1.01234567890123456789, 123456789e-10 into outfile %q", outfile)) cmpAndRm(`10000000000 1e20 123456700 0.123 1.01234567890123456789 0.0123456789 -`, outfile, c) +`, outfile, t) } -func (s *testSuite1) TestDeliminators(c *C) { +func TestDeliminators(t *testing.T) { outfile := randomSelectFilePath("TestDeliminators") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE `tx` (`a` varbinary(20) DEFAULT NULL,`b` int DEFAULT NULL)") err := tk.ExecToErr(fmt.Sprintf("select * from `tx` into outfile %q fields enclosed by '\"\"'", outfile)) // enclosed by must be a single character - c.Check(err, NotNil) - c.Assert(strings.Contains(err.Error(), "Field separator argument is not what is expected"), IsTrue, Commentf("err: %v", err)) + require.Error(t, err) + require.Truef(t, strings.Contains(err.Error(), "Field separator argument is not what is expected"), "err: %v", err) err = tk.ExecToErr(fmt.Sprintf("select * from `tx` into outfile %q fields escaped by 'gg'", outfile)) // so does escaped by - c.Check(err, NotNil) - c.Assert(strings.Contains(err.Error(), "Field separator argument is not what is expected"), IsTrue, Commentf("err: %v", err)) + require.Error(t, err) + require.Truef(t, strings.Contains(err.Error(), "Field separator argument is not what is expected"), "err: %v", err) // since the above two test cases failed, it should not has outfile remained on disk _, err = os.Stat(outfile) - c.Check(os.IsNotExist(err), IsTrue, Commentf("err: %v", err)) + require.Truef(t, os.IsNotExist(err), "err: %v", err) tk.MustExec("insert into tx values (NULL, NULL);\n") tk.MustExec(fmt.Sprintf("select * from `tx` into outfile %q fields escaped by ''", outfile)) // if escaped by is set as empty, then NULL should not be escaped - cmpAndRm("NULL\tNULL\n", outfile, c) + cmpAndRm("NULL\tNULL\n", outfile, t) tk.MustExec("delete from tx") tk.MustExec("insert into tx values ('d\",\"e\",', 3), ('\\\\', 2)") tk.MustExec(fmt.Sprintf("select * from `tx` into outfile %q FIELDS TERMINATED BY ',' ENCLOSED BY '\"' LINES TERMINATED BY '\\n'", outfile)) // enclosed by character & escaped by characters should be escaped, no matter what - cmpAndRm("\"d\\\",\\\"e\\\",\",\"3\"\n\"\\\\\",\"2\"\n", outfile, c) + cmpAndRm("\"d\\\",\\\"e\\\",\",\"3\"\n\"\\\\\",\"2\"\n", outfile, t) tk.MustExec("delete from tx") tk.MustExec("insert into tx values ('a\tb', 1)") tk.MustExec(fmt.Sprintf("select * from `tx` into outfile %q FIELDS TERMINATED BY ',' ENCLOSED BY '\"' escaped by '\t' LINES TERMINATED BY '\\n'", outfile)) // enclosed by character & escaped by characters should be escaped, no matter what - cmpAndRm("\"a\t\tb\",\"1\"\n", outfile, c) + cmpAndRm("\"a\t\tb\",\"1\"\n", outfile, t) tk.MustExec("delete from tx") tk.MustExec(`insert into tx values ('d","e",', 1)`) @@ -201,34 +211,34 @@ func (s *testSuite1) TestDeliminators(c *C) { tk.MustExec(`insert into tx values (null, 4)`) tk.MustExec(fmt.Sprintf("select * from `tx` into outfile %q FIELDS TERMINATED BY ',' ENCLOSED BY '\"' LINES TERMINATED BY '\\n'", outfile)) // line terminator will be escaped - cmpAndRm("\"d\\\",\\\"e\\\",\",\"1\"\n"+"\"\\0\",\"2\"\n"+"\"\r\\\n\b\032\t\",\"3\"\n"+"\\N,\"4\"\n", outfile, c) + cmpAndRm("\"d\\\",\\\"e\\\",\",\"1\"\n"+"\"\\0\",\"2\"\n"+"\"\r\\\n\b\032\t\",\"3\"\n"+"\\N,\"4\"\n", outfile, t) tk.MustExec("create table tb (s char(10), b bit(48), bb blob(6))") tk.MustExec("insert into tb values ('\\0\\b\\n\\r\\t\\Z', _binary '\\0\\b\\n\\r\\t\\Z', unhex('00080A0D091A'))") tk.MustExec(fmt.Sprintf("select * from tb into outfile %q", outfile)) // bit type won't be escaped (verified on MySQL) - cmpAndRm("\\0\b\\\n\r\\\t\032\t"+"\000\b\n\r\t\032\t"+"\\0\b\\\n\r\\\t\032\n", outfile, c) + cmpAndRm("\\0\b\\\n\r\\\t\032\t"+"\000\b\n\r\t\032\t"+"\\0\b\\\n\r\\\t\032\n", outfile, t) tk.MustExec("create table zero (a varchar(10), b varchar(10), c varchar(10))") tk.MustExec("insert into zero values (unhex('00'), _binary '\\0', '\\0')") tk.MustExec(fmt.Sprintf("select * from zero into outfile %q", outfile)) // zero will always be escaped - cmpAndRm("\\0\t\\0\t\\0\n", outfile, c) + cmpAndRm("\\0\t\\0\t\\0\n", outfile, t) tk.MustExec(fmt.Sprintf("select * from zero into outfile %q fields enclosed by '\"'", outfile)) // zero will always be escaped, including when being enclosed - cmpAndRm("\"\\0\"\t\"\\0\"\t\"\\0\"\n", outfile, c) + cmpAndRm("\"\\0\"\t\"\\0\"\t\"\\0\"\n", outfile, t) tk.MustExec("create table tt (a char(10), b char(10), c char(10))") tk.MustExec("insert into tt values ('abcd', 'abcd', 'abcd')") tk.MustExec(fmt.Sprintf("select * from tt into outfile %q fields terminated by 'a-' lines terminated by 'b--'", outfile)) // when not escaped, the first character of both terminators will be escaped - cmpAndRm("\\a\\bcda-\\a\\bcda-\\a\\bcdb--", outfile, c) + cmpAndRm("\\a\\bcda-\\a\\bcda-\\a\\bcdb--", outfile, t) tk.MustExec(fmt.Sprintf("select * from tt into outfile %q fields terminated by 'a-' enclosed by '\"' lines terminated by 'b--'", outfile)) // when escaped, only line terminator's first character will be escaped - cmpAndRm("\"a\\bcd\"a-\"a\\bcd\"a-\"a\\bcd\"b--", outfile, c) + cmpAndRm("\"a\\bcd\"a-\"a\\bcd\"a-\"a\\bcd\"b--", outfile, t) } -func (s *testSuite1) TestDumpReal(c *C) { +func TestDumpReal(t *testing.T) { cases := []struct { val float64 dec int @@ -247,13 +257,15 @@ func (s *testSuite1) TestDumpReal(c *C) { tp := types.NewFieldType(mysql.TypeDouble) tp.Decimal = testCase.dec _, buf := executor.DumpRealOutfile(nil, nil, testCase.val, tp) - c.Assert(string(buf), Equals, testCase.result) + require.Equal(t, testCase.result, string(buf)) } } -func (s *testSuite1) TestEscapeType(c *C) { +func TestEscapeType(t *testing.T) { outfile := randomSelectFilePath("TestEscapeType") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(`create table t ( @@ -268,12 +280,14 @@ func (s *testSuite1) TestEscapeType(c *C) { tk.MustExec(fmt.Sprintf("select * from t into outfile '%v' fields terminated by ',' escaped by '1'", outfile)) cmpAndRm(`1,1,11,11,{"key": 11},11,11 -`, outfile, c) +`, outfile, t) } -func (s *testSuite1) TestYearType(c *C) { +func TestYearType(t *testing.T) { outfile := randomSelectFilePath("TestYearType") - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(time1 year(4) default '2030');") @@ -281,5 +295,5 @@ func (s *testSuite1) TestYearType(c *C) { tk.MustExec("insert into t values ();") tk.MustExec(fmt.Sprintf("select * from t into outfile '%v' fields terminated by ',' optionally enclosed by '\"' lines terminated by '\\n';", outfile)) - cmpAndRm("2010\n2011\n2012\n2030\n", outfile, c) + cmpAndRm("2010\n2011\n2012\n2030\n", outfile, t) } diff --git a/executor/seqtest/main_test.go b/executor/seqtest/main_test.go index 737a21822d083..39253aff765fb 100644 --- a/executor/seqtest/main_test.go +++ b/executor/seqtest/main_test.go @@ -30,7 +30,7 @@ func TestMain(m *testing.M) { }) opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/pingcap/tidb/executor.readProjectionInput"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 8019843b50f84..f203ebbfd2df7 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -35,7 +35,6 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" ddltestutil "github.com/pingcap/tidb/ddl/testutil" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" @@ -51,7 +50,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" @@ -62,7 +60,7 @@ import ( func TestEarlyClose(t *testing.T) { var cluster testutils.Cluster - store, clean := testkit.CreateMockStore(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { mockstore.BootstrapWithSingleStore(c) cluster = c })) @@ -81,7 +79,6 @@ func TestEarlyClose(t *testing.T) { tk.MustExec("insert earlyclose values " + strings.Join(values, ",")) // Get table ID for split. - dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("earlyclose")) require.NoError(t, err) @@ -1196,8 +1193,6 @@ func TestCoprocessorPriority(t *testing.T) { } func TestShowForNewCollations(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1223,9 +1218,6 @@ func TestForbidUnsupportedCollations(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(t, store) mustGetUnsupportedCollation := func(sql string, coll string) { tk.MustGetErrMsg(sql, fmt.Sprintf("[ddl:1273]Unsupported collation when new collation is enabled: '%s'", coll)) diff --git a/executor/set_config.go b/executor/set_config.go index 6faf9deec08e1..2d8f4d7e2e385 100644 --- a/executor/set_config.go +++ b/executor/set_config.go @@ -180,7 +180,7 @@ func ConvertConfigItem2JSON(ctx sessionctx.Context, key string, val expression.E var s string s, isNull, err = val.EvalString(ctx, chunk.Row{}) if err == nil && !isNull { - str = fmt.Sprintf(`"%s"`, s) + str = fmt.Sprintf("%q", s) } case types.ETInt: var i int64 diff --git a/executor/set_test.go b/executor/set_test.go index f1991c52f37b0..79219259fbd48 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -143,10 +143,10 @@ func (s *testSerialSuite1) TestSetVar(c *C) { c.Assert(charset, Equals, "utf8") c.Assert(collation, Equals, "utf8_bin") - tk.MustExec("set names latin1 collate latin1_swedish_ci") + tk.MustExec("set names latin1 collate latin1_bin") charset, collation = vars.GetCharsetInfo() c.Assert(charset, Equals, "latin1") - c.Assert(collation, Equals, "latin1_swedish_ci") + c.Assert(collation, Equals, "latin1_bin") tk.MustExec("set names utf8 collate default") charset, collation = vars.GetCharsetInfo() @@ -1517,6 +1517,7 @@ func (s *testSuite5) TestSetClusterConfigJSONData(c *C) { {&expression.Constant{Value: types.NewDatum(nil), RetType: types.NewFieldType(mysql.TypeLonglong)}, "", false}, {&expression.Constant{RetType: types.NewFieldType(mysql.TypeJSON)}, "", false}, // unsupported type {nil, "", false}, + {&expression.Constant{Value: types.NewDatum(`["no","no","lz4","lz4","lz4","zstd","zstd"]`), RetType: types.NewFieldType(mysql.TypeString)}, `{"k":"[\"no\",\"no\",\"lz4\",\"lz4\",\"lz4\",\"zstd\",\"zstd\"]"}`, true}, } ctx := mock.NewContext() diff --git a/executor/show.go b/executor/show.go index 1fef2db83c915..349a5ad307de5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -343,11 +343,7 @@ func (e *ShowExec) fetchShowBind() error { func (e *ShowExec) fetchShowEngines(ctx context.Context) error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, `SELECT * FROM information_schema.engines`) - if err != nil { - return errors.Trace(err) - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT * FROM information_schema.engines`) if err != nil { return errors.Trace(err) } @@ -430,14 +426,32 @@ func (e *ShowExec) fetchShowTables() error { return ErrBadDB.GenWithStackByArgs(e.DBName) } // sort for tables - tableNames := make([]string, 0, len(e.is.SchemaTables(e.DBName))) + schemaTables := e.is.SchemaTables(e.DBName) + tableNames := make([]string, 0, len(schemaTables)) activeRoles := e.ctx.GetSessionVars().ActiveRoles - var tableTypes = make(map[string]string) - for _, v := range e.is.SchemaTables(e.DBName) { + var ( + tableTypes = make(map[string]string) + fieldPatternsRegexp *regexp.Regexp + FieldFilterEnable bool + fieldFilter string + ) + if e.Extractor != nil { + extractor := (e.Extractor).(*plannercore.ShowTablesTableExtractor) + if extractor.FieldPatterns != "" { + fieldPatternsRegexp = regexp.MustCompile(extractor.FieldPatterns) + } + FieldFilterEnable = extractor.Field != "" + fieldFilter = extractor.Field + } + for _, v := range schemaTables { // Test with mysql.AllPrivMask means any privilege would be OK. // TODO: Should consider column privileges, which also make a table visible. if checker != nil && !checker.RequestVerification(activeRoles, e.DBName.O, v.Meta().Name.O, "", mysql.AllPrivMask) { continue + } else if FieldFilterEnable && v.Meta().Name.L != fieldFilter { + continue + } else if fieldPatternsRegexp != nil && !fieldPatternsRegexp.MatchString(v.Meta().Name.L) { + continue } tableNames = append(tableNames, v.Meta().Name.O) if v.Meta().IsView() { @@ -474,17 +488,6 @@ func (e *ShowExec) fetchShowTableStatus(ctx context.Context) error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, `SELECT - table_name, engine, version, row_format, table_rows, - avg_row_length, data_length, max_data_length, index_length, - data_free, auto_increment, create_time, update_time, check_time, - table_collation, IFNULL(checksum,''), create_options, table_comment - FROM information_schema.tables - WHERE lower(table_schema)=%? ORDER BY table_name`, e.DBName.L) - if err != nil { - return errors.Trace(err) - } - var snapshot uint64 txn, err := e.ctx.Txn(false) if err != nil { @@ -497,7 +500,13 @@ func (e *ShowExec) fetchShowTableStatus(ctx context.Context) error { snapshot = e.ctx.GetSessionVars().SnapshotTS } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt, sqlexec.ExecOptionWithSnapshot(snapshot)) + rows, _, err := exec.ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionWithSnapshot(snapshot)}, + `SELECT table_name, engine, version, row_format, table_rows, + avg_row_length, data_length, max_data_length, index_length, + data_free, auto_increment, create_time, update_time, check_time, + table_collation, IFNULL(checksum,''), create_options, table_comment + FROM information_schema.tables + WHERE lower(table_schema)=%? ORDER BY table_name`, e.DBName.L) if err != nil { return errors.Trace(err) } @@ -1423,11 +1432,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, strings.ToLower(hostName)) - if err != nil { - return errors.Trace(err) - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, strings.ToLower(hostName)) if err != nil { return errors.Trace(err) } @@ -1443,11 +1448,7 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error { authplugin = rows[0].GetString(0) } - stmt, err = exec.ParseWithParams(ctx, true, `SELECT Priv FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.GlobalPrivTable, userName, hostName) - if err != nil { - return errors.Trace(err) - } - rows, _, err = exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err = exec.ExecRestrictedSQL(ctx, nil, `SELECT Priv FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.GlobalPrivTable, userName, hostName) if err != nil { return errors.Trace(err) } diff --git a/executor/show_placement.go b/executor/show_placement.go index 164a5f9293a7c..acd6d9cccecfc 100644 --- a/executor/show_placement.go +++ b/executor/show_placement.go @@ -107,12 +107,7 @@ func (b *showPlacementLabelsResultBuilder) sortMapKeys(m map[string]interface{}) func (e *ShowExec) fetchShowPlacementLabels(ctx context.Context) error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, "SELECT DISTINCT LABEL FROM %n.%n", "INFORMATION_SCHEMA", infoschema.TableTiKVStoreStatus) - if err != nil { - return errors.Trace(err) - } - - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, "SELECT DISTINCT LABEL FROM %n.%n", "INFORMATION_SCHEMA", infoschema.TableTiKVStoreStatus) if err != nil { return errors.Trace(err) } diff --git a/executor/show_placement_labels_test.go b/executor/show_placement_labels_test.go index 3e3efcd865738..a6d17b6b5e203 100644 --- a/executor/show_placement_labels_test.go +++ b/executor/show_placement_labels_test.go @@ -16,18 +16,14 @@ package executor import ( gjson "encoding/json" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/types/json" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testShowPlacementLabelSuit{}) - -type testShowPlacementLabelSuit struct { -} - -func (s *testShowPlacementLabelSuit) TestShowPlacementLabelsBuilder(c *C) { +func TestShowPlacementLabelsBuilder(t *testing.T) { cases := []struct { stores [][]*helper.StoreLabel expects [][]interface{} @@ -57,9 +53,9 @@ func (s *testShowPlacementLabelSuit) TestShowPlacementLabelsBuilder(c *C) { b := &showPlacementLabelsResultBuilder{} toBinaryJSON := func(obj interface{}) (bj json.BinaryJSON) { d, err := gjson.Marshal(obj) - c.Assert(err, IsNil) + require.NoError(t, err) err = bj.UnmarshalJSON(d) - c.Assert(err, IsNil) + require.NoError(t, err) return } @@ -67,19 +63,19 @@ func (s *testShowPlacementLabelSuit) TestShowPlacementLabelsBuilder(c *C) { for _, store := range ca.stores { bj := toBinaryJSON(store) err := b.AppendStoreLabels(bj) - c.Assert(err, IsNil) + require.NoError(t, err) } rows, err := b.BuildRows() - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, len(ca.expects)) + require.NoError(t, err) + require.Equal(t, len(ca.expects), len(rows)) for idx, expect := range ca.expects { row := rows[idx] bj := toBinaryJSON(expect[1]) - c.Assert(row[0].(string), Equals, expect[0].(string)) - c.Assert(row[1].(json.BinaryJSON).TypeCode, Equals, bj.TypeCode) - c.Assert(row[1].(json.BinaryJSON).Value, BytesEquals, bj.Value) + require.Equal(t, expect[0].(string), row[0].(string)) + require.Equal(t, bj.TypeCode, row[1].(json.BinaryJSON).TypeCode) + require.Equal(t, bj.Value, row[1].(json.BinaryJSON).Value) } } } diff --git a/executor/show_placement_test.go b/executor/show_placement_test.go index 7ed3e5732b217..61279c7adfc79 100644 --- a/executor/show_placement_test.go +++ b/executor/show_placement_test.go @@ -16,17 +16,19 @@ package executor_test import ( "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite5) TestShowPlacement(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacement(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3, t4, db2.t2") @@ -107,8 +109,10 @@ func (s *testSuite5) TestShowPlacement(c *C) { )) } -func (s *testSuite5) TestShowPlacementPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacementPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2,t3, db2.t1, db2.t3") tk.MustExec("drop database if exists db2") @@ -148,11 +152,8 @@ func (s *testSuite5) TestShowPlacementPrivilege(c *C) { tk.MustExec("create table db2.t3 (id int) PLACEMENT POLICY p1") defer tk.MustExec("drop table if exists db2.t3") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil)) // before grant tk1.MustQuery("show placement").Check(testkit.Rows( @@ -174,8 +175,10 @@ func (s *testSuite5) TestShowPlacementPrivilege(c *C) { )) } -func (s *testSuite5) TestShowPlacementForDB(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacementForDB(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop database if exists db2") @@ -191,7 +194,7 @@ func (s *testSuite5) TestShowPlacementForDB(c *C) { defer tk.MustExec("drop database if exists db2") err := tk.QueryToErr("show placement for database dbnoexist") - c.Assert(err.Error(), Equals, "[schema:1049]Unknown database 'dbnoexist'") + require.EqualError(t, err, "[schema:1049]Unknown database 'dbnoexist'") tk.MustQuery("show placement for database test").Check(testkit.Rows()) tk.MustQuery("show placement for database db2").Check(testkit.Rows( @@ -199,8 +202,10 @@ func (s *testSuite5) TestShowPlacementForDB(c *C) { )) } -func (s *testSuite5) TestShowPlacementForTableAndPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacementForTableAndPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop placement policy if exists p1") tk.MustExec("drop placement policy if exists p2") @@ -266,19 +271,21 @@ func (s *testSuite5) TestShowPlacementForTableAndPartition(c *C) { // not exists err := tk.ExecToErr("show placement for table tn") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.tn' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.tn' doesn't exist") err = tk.ExecToErr("show placement for table dbn.t1") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'dbn.t1' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'dbn.t1' doesn't exist") err = tk.ExecToErr("show placement for table tn partition pn") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.tn' doesn't exist") + require.EqualError(t, err, "[schema:1146]Table 'test.tn' doesn't exist") err = tk.QueryToErr("show placement for table t1 partition pn") - c.Assert(err.Error(), Equals, "[table:1735]Unknown partition 'pn' in table 't1'") + require.EqualError(t, err, "[table:1735]Unknown partition 'pn' in table 't1'") err = tk.QueryToErr("show placement for table t4 partition pn") - c.Assert(err.Error(), Equals, "[table:1735]Unknown partition 'pn' in table 't4'") + require.EqualError(t, err, "[table:1735]Unknown partition 'pn' in table 't4'") } -func (s *testSuite5) TestShowPlacementForDBPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacementForDBPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists db2.t1") @@ -305,11 +312,8 @@ func (s *testSuite5) TestShowPlacementForDBPrivilege(c *C) { tk.MustExec("create table db2.t1 (id int) PLACEMENT POLICY p2") defer tk.MustExec("drop table db2.t1") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil)) privs := []string{ "all privileges on db2.*", @@ -323,8 +327,8 @@ func (s *testSuite5) TestShowPlacementForDBPrivilege(c *C) { } // before grant - err = tk1.QueryToErr("show placement for database db2") - c.Assert(err.Error(), Equals, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "db2").Error()) + err := tk1.QueryToErr("show placement for database db2") + require.EqualError(t, err, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "db2").Error()) tk1.MustQuery("show placement").Check(testkit.Rows( "POLICY p1 PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" SCHEDULE=\"EVEN\" NULL", @@ -346,12 +350,12 @@ func (s *testSuite5) TestShowPlacementForDBPrivilege(c *C) { )) err = tk1.QueryToErr("show placement for database test") - c.Assert(err.Error(), Equals, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "test").Error()) + require.EqualError(t, err, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "test").Error()) // do revoke tk.MustExec(fmt.Sprintf("revoke %s from 'user1'@'%%'", priv)) err = tk1.QueryToErr("show placement for database db2") - c.Assert(err.Error(), Equals, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "db2").Error()) + require.EqualError(t, err, executor.ErrDBaccessDenied.GenWithStackByArgs("user1", "%", "db2").Error()) tk1.MustQuery("show placement").Check(testkit.Rows( "POLICY p1 PRIMARY_REGION=\"r1\" REGIONS=\"r1,r2\" SCHEDULE=\"EVEN\" NULL", @@ -360,8 +364,10 @@ func (s *testSuite5) TestShowPlacementForDBPrivilege(c *C) { } } -func (s *testSuite5) TestShowPlacementForTableAndPartitionPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPlacementForTableAndPartitionPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2,t3,t4,db2.t1") tk.MustExec("drop database if exists db2") @@ -399,27 +405,24 @@ func (s *testSuite5) TestShowPlacementForTableAndPartitionPrivilege(c *C) { tk.MustExec("create table db2.t1 (id int) PLACEMENT POLICY p2") defer tk.MustExec("drop table if exists db2.t1") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "user1", Hostname: "%"}, nil, nil)) // before grant - err = tk1.ExecToErr("show placement for table test.t1") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) + err := tk1.ExecToErr("show placement for table test.t1") + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) err = tk1.ExecToErr("show placement for table test.t1 partition p1") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) err = tk1.ExecToErr("show placement for table test.t2") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t2").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t2").Error()) err = tk1.ExecToErr("show placement for table test.t3") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t3").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t3").Error()) err = tk1.ExecToErr("show placement for table db2.t1") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) tk1.MustQuery("show placement").Check(testkit.Rows( "POLICY p1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" NULL", @@ -455,18 +458,18 @@ func (s *testSuite5) TestShowPlacementForTableAndPartitionPrivilege(c *C) { )) err = tk1.ExecToErr("show placement for table test.t2") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t2").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t2").Error()) err = tk1.ExecToErr("show placement for table test.t3") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t3").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t3").Error()) err = tk1.ExecToErr("show placement for table db2.t1") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) // do revoke tk.MustExec(fmt.Sprintf("revoke %s from 'user1'@'%%'", priv)) err = tk1.ExecToErr("show placement for table test.t1") - c.Assert(err.Error(), Equals, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) + require.EqualError(t, err, core.ErrTableaccessDenied.GenWithStackByArgs("SHOW", "user1", "%", "t1").Error()) tk1.MustQuery("show placement").Check(testkit.Rows( "POLICY p1 PRIMARY_REGION=\"cn-east-1\" REGIONS=\"cn-east-1,cn-east-2\" SCHEDULE=\"EVEN\" NULL", diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index f48fe9988649d..cbb5465ec5c9a 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -19,7 +19,6 @@ import ( "testing" "time" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" @@ -311,7 +310,7 @@ func TestShowStatsExtended(t *testing.T) { } func TestShowColumnStatsUsage(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -320,7 +319,6 @@ func TestShowColumnStatsUsage(t *testing.T) { tk.MustExec("create table t1 (a int, b int, index idx_a_b(a, b))") tk.MustExec("create table t2 (a int, b int) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue)") - dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() t1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) require.NoError(t, err) @@ -352,7 +350,7 @@ func TestShowHistogramsInFlight(t *testing.T) { tk.MustExec("use test") result := tk.MustQuery("show histograms_in_flight") rows := result.Rows() - require.Equal(t, len(rows), 1) + require.Len(t, rows, 1) require.Equal(t, rows[0][0], "0") } diff --git a/executor/show_test.go b/executor/show_test.go index 71faf1c1925b4..88e25dd1c5e4e 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -18,11 +18,9 @@ import ( "context" "fmt" "strings" + "testing" - "github.com/pingcap/check" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/auth" @@ -32,26 +30,25 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -func (s *testSuite5) TestShowVisibility(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowVisibility(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database showdatabase") tk.MustExec("use showdatabase") tk.MustExec("create table t1 (id int)") tk.MustExec("create table t2 (id int)") tk.MustExec(`create user 'show'@'%'`) - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "show", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "show", Hostname: "%"}, nil, nil)) // No ShowDatabases privilege, this user would see nothing except INFORMATION_SCHEMA. tk.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA")) @@ -71,14 +68,16 @@ func (s *testSuite5) TestShowVisibility(c *C) { // Grant any global privilege would make show databases available. tk.MustExec(`grant CREATE on *.* to 'show'@'%'`) rows := tk1.MustQuery("show databases").Rows() - c.Assert(len(rows), GreaterEqual, 2) // At least INFORMATION_SCHEMA and showdatabase + require.GreaterOrEqual(t, len(rows), 2) tk.MustExec(`drop user 'show'@'%'`) tk.MustExec("drop database showdatabase") } -func (s *testSuite5) TestShowDatabasesInfoSchemaFirst(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowDatabasesInfoSchemaFirst(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA")) tk.MustExec(`create user 'show'@'%'`) @@ -87,11 +86,8 @@ func (s *testSuite5) TestShowDatabasesInfoSchemaFirst(c *C) { tk.MustExec(`grant select on AAAA.* to 'show'@'%'`) tk.MustExec(`grant select on BBBB.* to 'show'@'%'`) - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "show", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "show", Hostname: "%"}, nil, nil)) tk1.MustQuery("show databases").Check(testkit.Rows("INFORMATION_SCHEMA", "AAAA", "BBBB")) tk.MustExec(`drop user 'show'@'%'`) @@ -99,24 +95,26 @@ func (s *testSuite5) TestShowDatabasesInfoSchemaFirst(c *C) { tk.MustExec(`drop database BBBB`) } -func (s *testSuite5) TestShowWarnings(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowWarnings(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `create table if not exists show_warnings (a int)` tk.MustExec(testSQL) tk.MustExec("set @@sql_mode=''") tk.MustExec("insert show_warnings values ('a')") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: 'a'")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: 'a'")) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) // Test Warning level 'Error' testSQL = `create table show_warnings (a int)` _, _ = tk.Exec(testSQL) // FIXME: Table 'test.show_warnings' already exists - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Error|1050|Table 'test.show_warnings' already exists")) tk.MustQuery("select @@error_count").Check(testutil.RowsWithSep("|", "1")) @@ -125,15 +123,17 @@ func (s *testSuite5) TestShowWarnings(c *C) { tk.MustExec(testSQL) testSQL = `create table if not exists show_warnings_2 like show_warnings` _, err := tk.Exec(testSQL) - c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + require.NoError(t, err) + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1050|Table 'test.show_warnings_2' already exists")) tk.MustQuery("select @@warning_count").Check(testutil.RowsWithSep("|", "1")) tk.MustQuery("select @@warning_count").Check(testutil.RowsWithSep("|", "0")) } -func (s *testSuite5) TestShowErrors(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowErrors(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `create table if not exists show_errors (a int)` tk.MustExec(testSQL) @@ -144,17 +144,19 @@ func (s *testSuite5) TestShowErrors(c *C) { tk.MustQuery("show errors").Check(testutil.RowsWithSep("|", "Error|1050|Table 'test.show_errors' already exists")) } -func (s *testSuite5) TestShowWarningsForExprPushdown(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowWarningsForExprPushdown(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `create table if not exists show_warnings_expr_pushdown (a int, value date)` tk.MustExec(testSQL) // create tiflash replica { - is := domain.GetDomain(tk.Se).InfoSchema() + is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "show_warnings_expr_pushdown" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -166,57 +168,53 @@ func (s *testSuite5) TestShowWarningsForExprPushdown(c *C) { } tk.MustExec("set tidb_allow_mpp=0") tk.MustExec("explain select * from show_warnings_expr_pushdown t where md5(value) = '2020-01-01'") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) + require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.")) tk.MustExec("explain select max(md5(value)) from show_warnings_expr_pushdown group by a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) + require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because arguments of AggFunc `max` contains unsupported exprs")) tk.MustExec("explain select max(a) from show_warnings_expr_pushdown group by md5(value)") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) + require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because groupByItems contain unsupported exprs")) tk.MustExec("set tidb_opt_distinct_agg_push_down=0") tk.MustExec("explain select max(distinct a) from show_warnings_expr_pushdown group by value") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(0)) + require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) // tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|Aggregation can not be pushed to storage layer in non-mpp mode because it contains agg function with distinct")) } -func (s *testSuite5) TestShowGrantsPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowGrantsPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create user show_grants") tk.MustExec("show grants for show_grants") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "show_grants", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se - err = tk1.QueryToErr("show grants for root") - c.Assert(err.Error(), Equals, executor.ErrDBaccessDenied.GenWithStackByArgs("show_grants", "%", mysql.SystemDB).Error()) + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "show_grants", Hostname: "%"}, nil, nil)) + err := tk1.QueryToErr("show grants for root") + require.EqualError(t, executor.ErrDBaccessDenied.GenWithStackByArgs("show_grants", "%", mysql.SystemDB), err.Error()) // Test show grants for user with auth host name `%`. - tk2 := testkit.NewTestKit(c, s.store) - se2, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se2.Auth(&auth.UserIdentity{Username: "show_grants", Hostname: "127.0.0.1", AuthUsername: "show_grants", AuthHostname: "%"}, nil, nil), IsTrue) - tk2.Se = se2 + tk2 := testkit.NewTestKit(t, store) + require.True(t, tk2.Session().Auth(&auth.UserIdentity{Username: "show_grants", Hostname: "127.0.0.1", AuthUsername: "show_grants", AuthHostname: "%"}, nil, nil)) tk2.MustQuery("show grants") } -func (s *testSuite5) TestShowStatsPrivilege(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowStatsPrivilege(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create user show_stats") - tk1 := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "show_stats", Hostname: "%"}, nil, nil), IsTrue) - tk1.Se = se + tk1 := testkit.NewTestKit(t, store) + + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "show_stats", Hostname: "%"}, nil, nil)) eqErr := plannercore.ErrDBaccessDenied.GenWithStackByArgs("show_stats", "%", mysql.SystemDB) - _, err = tk1.Exec("show stats_meta") - c.Assert(err.Error(), Equals, eqErr.Error()) + _, err := tk1.Exec("show stats_meta") + require.EqualError(t, err, eqErr.Error()) _, err = tk1.Exec("SHOW STATS_BUCKETS") - c.Assert(err.Error(), Equals, eqErr.Error()) + require.EqualError(t, err, eqErr.Error()) _, err = tk1.Exec("SHOW STATS_HEALTHY") - c.Assert(err.Error(), Equals, eqErr.Error()) + require.EqualError(t, err, eqErr.Error()) _, err = tk1.Exec("SHOW STATS_HISTOGRAMS") - c.Assert(err.Error(), Equals, eqErr.Error()) + require.EqualError(t, err, eqErr.Error()) tk.MustExec("grant select on mysql.* to show_stats") tk1.MustExec("show stats_meta") tk1.MustExec("SHOW STATS_BUCKETS") @@ -224,55 +222,53 @@ func (s *testSuite5) TestShowStatsPrivilege(c *C) { tk1.MustExec("SHOW STATS_HISTOGRAMS") } -func (s *testSuite5) TestIssue18878(c *C) { - tk := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "127.0.0.1", AuthHostname: "%"}, nil, nil), IsTrue) - tk.Se = se +func TestIssue18878(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "127.0.0.1", AuthHostname: "%"}, nil, nil)) tk.MustQuery("select user()").Check(testkit.Rows("root@127.0.0.1")) tk.MustQuery("show grants") tk.MustQuery("select user()").Check(testkit.Rows("root@127.0.0.1")) - err = tk.QueryToErr("show grants for root@127.0.0.1") - c.Assert(err.Error(), Equals, privileges.ErrNonexistingGrant.FastGenByArgs("root", "127.0.0.1").Error()) + err := tk.QueryToErr("show grants for root@127.0.0.1") + require.Equal(t, privileges.ErrNonexistingGrant.FastGenByArgs("root", "127.0.0.1").Error(), err.Error()) err = tk.QueryToErr("show grants for root@localhost") - c.Assert(err.Error(), Equals, privileges.ErrNonexistingGrant.FastGenByArgs("root", "localhost").Error()) + require.Equal(t, privileges.ErrNonexistingGrant.FastGenByArgs("root", "localhost").Error(), err.Error()) err = tk.QueryToErr("show grants for root@1.1.1.1") - c.Assert(err.Error(), Equals, privileges.ErrNonexistingGrant.FastGenByArgs("root", "1.1.1.1").Error()) + require.Equal(t, privileges.ErrNonexistingGrant.FastGenByArgs("root", "1.1.1.1").Error(), err.Error()) tk.MustExec("create user `show_grants`@`127.0.%`") err = tk.QueryToErr("show grants for `show_grants`@`127.0.0.1`") - c.Assert(err.Error(), Equals, privileges.ErrNonexistingGrant.FastGenByArgs("show_grants", "127.0.0.1").Error()) + require.Equal(t, privileges.ErrNonexistingGrant.FastGenByArgs("show_grants", "127.0.0.1").Error(), err.Error()) tk.MustQuery("show grants for `show_grants`@`127.0.%`") } -func (s *testSuite5) TestIssue17794(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue17794(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE USER 'root'@'8.8.%'") - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "9.9.9.9", AuthHostname: "%"}, nil, nil), IsTrue) - tk.Se = se - - tk1 := testkit.NewTestKit(c, s.store) - se1, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se1.Auth(&auth.UserIdentity{Username: "root", Hostname: "8.8.8.8", AuthHostname: "8.8.%"}, nil, nil), IsTrue) - tk1.Se = se1 + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "9.9.9.9", AuthHostname: "%"}, nil, nil)) + tk1 := testkit.NewTestKit(t, store) + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "8.8.8.8", AuthHostname: "8.8.%"}, nil, nil)) tk.MustQuery("show grants").Check(testkit.Rows("GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION")) tk1.MustQuery("show grants").Check(testkit.Rows("GRANT USAGE ON *.* TO 'root'@'8.8.%'")) } -func (s *testSuite5) TestIssue3641(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue3641(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) _, err := tk.Exec("show tables;") - c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) + require.Equal(t, plannercore.ErrNoDB.Error(), err.Error()) _, err = tk.Exec("show table status;") - c.Assert(err.Error(), Equals, plannercore.ErrNoDB.Error()) + require.Equal(t, plannercore.ErrNoDB.Error(), err.Error()) } -func (s *testSuite5) TestIssue10549(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue10549(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DATABASE newdb;") tk.MustExec("CREATE ROLE 'app_developer';") tk.MustExec("GRANT ALL ON newdb.* TO 'app_developer';") @@ -280,35 +276,39 @@ func (s *testSuite5) TestIssue10549(c *C) { tk.MustExec("GRANT 'app_developer' TO 'dev';") tk.MustExec("SET DEFAULT ROLE app_developer TO 'dev';") - c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "dev", Hostname: "%", AuthUsername: "dev", AuthHostname: "%"}, nil, nil), IsTrue) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "dev", Hostname: "%", AuthUsername: "dev", AuthHostname: "%"}, nil, nil)) tk.MustQuery("SHOW DATABASES;").Check(testkit.Rows("INFORMATION_SCHEMA", "newdb")) tk.MustQuery("SHOW GRANTS;").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT ALL PRIVILEGES ON newdb.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'")) tk.MustQuery("SHOW GRANTS FOR CURRENT_USER").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT ALL PRIVILEGES ON newdb.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'")) tk.MustQuery("SHOW GRANTS FOR dev").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'")) } -func (s *testSuite5) TestIssue11165(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue11165(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE ROLE 'r_manager';") tk.MustExec("CREATE USER 'manager'@'localhost';") tk.MustExec("GRANT 'r_manager' TO 'manager'@'localhost';") - c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "manager", Hostname: "localhost", AuthUsername: "manager", AuthHostname: "localhost"}, nil, nil), IsTrue) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "manager", Hostname: "localhost", AuthUsername: "manager", AuthHostname: "localhost"}, nil, nil)) tk.MustExec("SET DEFAULT ROLE ALL TO 'manager'@'localhost';") tk.MustExec("SET DEFAULT ROLE NONE TO 'manager'@'localhost';") tk.MustExec("SET DEFAULT ROLE 'r_manager' TO 'manager'@'localhost';") } // TestShow2 is moved from session_test -func (s *testSuite5) TestShow2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShow2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set global autocommit=0") - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustQuery("show global variables where variable_name = 'autocommit'").Check(testkit.Rows("autocommit OFF")) tk.MustExec("set global autocommit = 1") - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustQuery("show global variables where variable_name = 'autocommit'").Check(testkit.Rows("autocommit ON")) // TODO: Specifying the charset for national char/varchar should not be supported. @@ -403,14 +403,13 @@ func (s *testSuite5) TestShow2(c *C) { tk.MustQuery("SHOW FULL TABLES in information_schema like 'VIEWS'").Check(testkit.Rows("VIEWS SYSTEM VIEW")) tk.MustQuery("SHOW FULL TABLES in metrics_schema like 'uptime'").Check(testkit.Rows("uptime SYSTEM VIEW")) - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() + is := dom.InfoSchema() tblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) createTime := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format("2006-01-02 15:04:05") // The Hostname is the actual host - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) r := tk.MustQuery("show table status from test like 't'") r.Check(testkit.Rows(fmt.Sprintf("t InnoDB 10 Compact 0 0 0 0 0 0 %s utf8mb4_bin 注释", createTime))) @@ -424,8 +423,10 @@ func (s *testSuite5) TestShow2(c *C) { tk.MustQuery("show grants for current_user").Check(testkit.Rows(`GRANT ALL PRIVILEGES ON *.* TO 'root'@'%' WITH GRANT OPTION`)) } -func (s *testSuite5) TestShowCreateUser(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateUser(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // Create a new user. tk.MustExec(`CREATE USER 'test_show_create_user'@'%' IDENTIFIED BY 'root';`) tk.MustQuery("show create user 'test_show_create_user'@'%'"). @@ -437,57 +438,56 @@ func (s *testSuite5) TestShowCreateUser(c *C) { // Case: the user exists but the host portion doesn't match err := tk.QueryToErr("show create user 'test_show_create_user'@'asdf';") - c.Assert(err.Error(), Equals, executor.ErrCannotUser.GenWithStackByArgs("SHOW CREATE USER", "'test_show_create_user'@'asdf'").Error()) + require.Equal(t, executor.ErrCannotUser.GenWithStackByArgs("SHOW CREATE USER", "'test_show_create_user'@'asdf'").Error(), err.Error()) // Case: a user that doesn't exist err = tk.QueryToErr("show create user 'aaa'@'localhost';") - c.Assert(err.Error(), Equals, executor.ErrCannotUser.GenWithStackByArgs("SHOW CREATE USER", "'aaa'@'localhost'").Error()) + require.Equal(t, executor.ErrCannotUser.GenWithStackByArgs("SHOW CREATE USER", "'aaa'@'localhost'").Error(), err.Error()) - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "127.0.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, nil) - rows := tk.MustQuery("show create user current_user") - rows.Check(testkit.Rows("CREATE USER 'root'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "127.0.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, nil) + tk.MustQuery("show create user current_user"). + Check(testkit.Rows("CREATE USER 'root'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) - rows = tk.MustQuery("show create user current_user()") - rows.Check(testkit.Rows("CREATE USER 'root'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) + tk.MustQuery("show create user current_user()"). + Check(testkit.Rows("CREATE USER 'root'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) tk.MustExec("create user 'check_priv'") // "show create user" for other user requires the SELECT privilege on mysql database. - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use mysql") - succ := tk1.Se.Auth(&auth.UserIdentity{Username: "check_priv", Hostname: "127.0.0.1", AuthUsername: "test_show", AuthHostname: "asdf"}, nil, nil) - c.Assert(succ, IsTrue) + succ := tk1.Session().Auth(&auth.UserIdentity{Username: "check_priv", Hostname: "127.0.0.1", AuthUsername: "test_show", AuthHostname: "asdf"}, nil, nil) + require.True(t, succ) err = tk1.QueryToErr("show create user 'root'@'%'") - c.Assert(err, NotNil) + require.Error(t, err) // "show create user" for current user doesn't check privileges. - rows = tk1.MustQuery("show create user current_user") - rows.Check(testkit.Rows("CREATE USER 'check_priv'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) + tk1.MustQuery("show create user current_user"). + Check(testkit.Rows("CREATE USER 'check_priv'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) // Creating users with `IDENTIFIED WITH 'caching_sha2_password'` tk.MustExec("CREATE USER 'sha_test'@'%' IDENTIFIED WITH 'caching_sha2_password' BY 'temp_passwd'") // Compare only the start of the output as the salt changes every time. - rows = tk.MustQuery("SHOW CREATE USER 'sha_test'@'%'") - c.Assert(rows.Rows()[0][0].(string)[:78], check.Equals, "CREATE USER 'sha_test'@'%' IDENTIFIED WITH 'caching_sha2_password' AS '$A$005$") - + rows := tk.MustQuery("SHOW CREATE USER 'sha_test'@'%'") + require.Equal(t, "CREATE USER 'sha_test'@'%' IDENTIFIED WITH 'caching_sha2_password' AS '$A$005$", rows.Rows()[0][0].(string)[:78]) // Creating users with `IDENTIFIED WITH 'auth-socket'` tk.MustExec("CREATE USER 'sock'@'%' IDENTIFIED WITH 'auth_socket'") // Compare only the start of the output as the salt changes every time. rows = tk.MustQuery("SHOW CREATE USER 'sock'@'%'") - c.Assert(rows.Rows()[0][0].(string), check.Equals, "CREATE USER 'sock'@'%' IDENTIFIED WITH 'auth_socket' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK") - + require.Equal(t, "CREATE USER 'sock'@'%' IDENTIFIED WITH 'auth_socket' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK", rows.Rows()[0][0].(string)) tk.MustExec("CREATE USER 'sock2'@'%' IDENTIFIED WITH 'auth_socket' AS 'sock3'") // Compare only the start of the output as the salt changes every time. rows = tk.MustQuery("SHOW CREATE USER 'sock2'@'%'") - c.Assert(rows.Rows()[0][0].(string), check.Equals, "CREATE USER 'sock2'@'%' IDENTIFIED WITH 'auth_socket' AS 'sock3' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK") + require.Equal(t, "CREATE USER 'sock2'@'%' IDENTIFIED WITH 'auth_socket' AS 'sock3' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK", rows.Rows()[0][0].(string)) } -func (s *testSuite5) TestUnprivilegedShow(c *C) { - - tk := testkit.NewTestKit(c, s.store) +func TestUnprivilegedShow(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE DATABASE testshow") tk.MustExec("USE testshow") tk.MustExec("CREATE TABLE t1 (a int)") @@ -495,66 +495,69 @@ func (s *testSuite5) TestUnprivilegedShow(c *C) { tk.MustExec(`CREATE USER 'lowprivuser'`) // no grants - tk.Se.Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) rs, err := tk.Exec("SHOW TABLE STATUS FROM testshow") - c.Assert(err, IsNil) - c.Assert(rs, NotNil) + require.NoError(t, err) + require.NotNil(t, rs) - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) tk.MustExec("GRANT ALL ON testshow.t1 TO 'lowprivuser'") - tk.Se.Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "lowprivuser", Hostname: "192.168.0.1", AuthUsername: "lowprivuser", AuthHostname: "%"}, nil, []byte("012345678901234567890")) - ctx := tk.Se.(sessionctx.Context) - is := domain.GetDomain(ctx).InfoSchema() + is := dom.InfoSchema() tblInfo, err := is.TableByName(model.NewCIStr("testshow"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) createTime := model.TSConvert2Time(tblInfo.Meta().UpdateTS).Format("2006-01-02 15:04:05") tk.MustQuery("show table status from testshow").Check(testkit.Rows(fmt.Sprintf("t1 InnoDB 10 Compact 0 0 0 0 0 0 %s utf8mb4_bin ", createTime))) } -func (s *testSuite5) TestCollation(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") rs, err := tk.Exec("show collation;") - c.Assert(err, IsNil) + require.NoError(t, err) fields := rs.Fields() - c.Assert(fields[0].Column.Tp, Equals, mysql.TypeVarchar) - c.Assert(fields[1].Column.Tp, Equals, mysql.TypeVarchar) - c.Assert(fields[2].Column.Tp, Equals, mysql.TypeLonglong) - c.Assert(fields[3].Column.Tp, Equals, mysql.TypeVarchar) - c.Assert(fields[4].Column.Tp, Equals, mysql.TypeVarchar) - c.Assert(fields[5].Column.Tp, Equals, mysql.TypeLonglong) + require.Equal(t, mysql.TypeVarchar, fields[0].Column.Tp) + require.Equal(t, mysql.TypeVarchar, fields[1].Column.Tp) + require.Equal(t, mysql.TypeLonglong, fields[2].Column.Tp) + require.Equal(t, mysql.TypeVarchar, fields[3].Column.Tp) + require.Equal(t, mysql.TypeVarchar, fields[4].Column.Tp) + require.Equal(t, mysql.TypeLonglong, fields[5].Column.Tp) } -func (s *testSuite5) TestShowTableStatus(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowTableStatus(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a bigint);`) - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "192.168.0.1", AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) // It's not easy to test the result contents because every time the test runs, "Create_time" changed. tk.MustExec("show table status;") rs, err := tk.Exec("show table status;") - c.Assert(err, IsNil) - c.Assert(rs, NotNil) - rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) + require.NoError(t, err) + require.NotNil(t, rs) + rows, err := session.GetRows4Test(context.Background(), tk.Session(), rs) + require.NoError(t, err) err = rs.Close() - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, 1) + require.NoError(t, err) + require.Equal(t, 1, len(rows)) for i := range rows { row := rows[i] - c.Assert(row.GetString(0), Equals, "t") - c.Assert(row.GetString(1), Equals, "InnoDB") - c.Assert(row.GetInt64(2), Equals, int64(10)) - c.Assert(row.GetString(3), Equals, "Compact") + require.Equal(t, "t", row.GetString(0)) + require.Equal(t, "InnoDB", row.GetString(1)) + require.Equal(t, int64(10), row.GetInt64(2)) + require.Equal(t, "Compact", row.GetString(3)) } tk.MustExec(`drop table if exists tp;`) tk.MustExec(`create table tp (a int) @@ -564,38 +567,40 @@ func (s *testSuite5) TestShowTableStatus(c *C) { partition p2 values less than (maxvalue) );`) rs, err = tk.Exec("show table status from test like 'tp';") - c.Assert(err, IsNil) - rows, err = session.GetRows4Test(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) - c.Assert(rows[0].GetString(16), Equals, "partitioned") + require.NoError(t, err) + rows, err = session.GetRows4Test(context.Background(), tk.Session(), rs) + require.NoError(t, err) + require.Equal(t, "partitioned", rows[0].GetString(16)) tk.MustExec("create database UPPER_CASE") tk.MustExec("use UPPER_CASE") tk.MustExec("create table t (i int)") rs, err = tk.Exec("show table status") - c.Assert(err, IsNil) - c.Assert(rs, NotNil) - rows, err = session.GetRows4Test(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) + require.NoError(t, err) + require.NotNil(t, rs) + rows, err = session.GetRows4Test(context.Background(), tk.Session(), rs) + require.NoError(t, err) err = rs.Close() - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, 1) + require.NoError(t, err) + require.Equal(t, 1, len(rows)) tk.MustExec("use upper_case") rs, err = tk.Exec("show table status") - c.Assert(err, IsNil) - c.Assert(rs, NotNil) - rows, err = session.GetRows4Test(context.Background(), tk.Se, rs) - c.Assert(err, IsNil) + require.NoError(t, err) + require.NotNil(t, rs) + rows, err = session.GetRows4Test(context.Background(), tk.Session(), rs) + require.NoError(t, err) err = rs.Close() - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, 1) + require.NoError(t, err) + require.Equal(t, 1, len(rows)) tk.MustExec("drop database UPPER_CASE") } -func (s *testSuite5) TestShowSlow(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowSlow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // The test result is volatile, because // 1. Slow queries is stored in domain, which may be affected by other tests. // 2. Collecting slow queries is a asynchronous process, check immediately may not get the expected result. @@ -607,25 +612,29 @@ func (s *testSuite5) TestShowSlow(c *C) { tk.MustQuery(`admin show slow top all 3`) } -func (s *testSuite5) TestShowOpenTables(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowOpenTables(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustQuery("show open tables") tk.MustQuery("show open tables in test") } -func (s *testSuite5) TestShowCreateViewDefiner(c *C) { - tk := testkit.NewTestKit(c, s.store) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%", AuthUsername: "root", AuthHostname: "%"}, nil, nil), IsTrue) - tk.Se = se +func TestShowCreateViewDefiner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%", AuthUsername: "root", AuthHostname: "%"}, nil, nil)) + tk.MustExec("use test") tk.MustExec("create or replace view v1 as select 1") tk.MustQuery("show create view v1").Check(testutil.RowsWithSep("|", "v1|CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `v1` (`1`) AS SELECT 1 AS `1`|utf8mb4|utf8mb4_bin")) tk.MustExec("drop view v1") } -func (s *testSuite5) TestShowCreateTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -910,12 +919,11 @@ func (s *testSuite5) TestShowCreateTable(c *C) { tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b char(10) as ('a'));") result := tk.MustQuery("show create table t;").Rows()[0][1] - c.Assert(result, Matches, `(?s).*GENERATED ALWAYS AS \(_utf8mb4'a'\).*`) + require.Regexp(t, `(?s).*GENERATED ALWAYS AS \(_utf8mb4'a'\).*`, result) tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b char(10) as (_utf8'a'));") result = tk.MustQuery("show create table t;").Rows()[0][1] - c.Assert(result, Matches, `(?s).*GENERATED ALWAYS AS \(_utf8'a'\).*`) - + require.Regexp(t, `(?s).*GENERATED ALWAYS AS \(_utf8'a'\).*`, result) // Test show list partition table tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec(`DROP TABLE IF EXISTS t`) @@ -1003,8 +1011,10 @@ func (s *testSuite5) TestShowCreateTable(c *C) { )) } -func (s *testSuite5) TestShowCreateTablePlacement(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateTablePlacement(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") defer tk.MustExec(`DROP TABLE IF EXISTS t`) @@ -1162,8 +1172,10 @@ func (s *testSuite5) TestShowCreateTablePlacement(c *C) { tk.MustExec(`DROP TABLE t`) } -func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateTableAutoRandom(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // Basic show create table. @@ -1229,8 +1241,10 @@ func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { } // TestAutoIdCache overrides testAutoRandomSuite to test auto id cache. -func (s *testAutoRandomSuite) TestAutoIdCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoIdCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1264,8 +1278,10 @@ func (s *testAutoRandomSuite) TestAutoIdCache(c *C) { )) } -func (s *testSuite5) TestShowCreateStmtIgnoreLocalTemporaryTables(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreateStmtIgnoreLocalTemporaryTables(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // SHOW CREATE VIEW ignores local temporary table with the same name @@ -1280,7 +1296,7 @@ func (s *testSuite5) TestShowCreateStmtIgnoreLocalTemporaryTables(c *C) { )) tk.MustExec("drop view v1") err := tk.ExecToErr("show create view v1") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) // SHOW CREATE SEQUENCE ignores local temporary table with the same name tk.MustExec("drop view if exists seq1") @@ -1291,16 +1307,17 @@ func (s *testSuite5) TestShowCreateStmtIgnoreLocalTemporaryTables(c *C) { )) tk.MustExec("drop sequence seq1") err = tk.ExecToErr("show create sequence seq1") - c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + require.True(t, infoschema.ErrTableNotExists.Equal(err)) } -func (s *testAutoRandomSuite) TestAutoRandomBase(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) +func TestAutoRandomBase(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange")) }() - - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@allow_auto_random_explicit_insert = true") tk.MustExec("use test") @@ -1328,8 +1345,10 @@ func (s *testAutoRandomSuite) TestAutoRandomBase(c *C) { )) } -func (s *testSuite5) TestAutoRandomWithLargeSignedShowTableRegions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAutoRandomWithLargeSignedShowTableRegions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database if not exists auto_random_db;") defer tk.MustExec("drop database if exists auto_random_db;") tk.MustExec("use auto_random_db;") @@ -1343,12 +1362,14 @@ func (s *testSuite5) TestAutoRandomWithLargeSignedShowTableRegions(c *C) { Check(testkit.Rows("1 1")) startKey := tk.MustQuery("show table t regions;").Rows()[1][1].(string) idx := strings.Index(startKey, "_r_") - c.Assert(idx == -1, IsFalse) - c.Assert(startKey[idx+3] == '-', IsFalse, Commentf("actual key: %s", startKey)) + require.False(t, idx == -1) + require.Falsef(t, startKey[idx+3] == '-', "actual key: %s", startKey) } -func (s *testSuite5) TestShowEscape(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowEscape(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists `t``abl\"e`") @@ -1376,19 +1397,23 @@ func (s *testSuite5) TestShowEscape(c *C) { tk.MustExec("set sql_mode=@old_sql_mode") } -func (s *testSuite5) TestShowBuiltin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowBuiltin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) res := tk.MustQuery("show builtins;") - c.Assert(res, NotNil) + require.NotNil(t, res) rows := res.Rows() - const builtinFuncNum = 274 - c.Assert(builtinFuncNum, Equals, len(rows)) - c.Assert("abs", Equals, rows[0][0].(string)) - c.Assert("yearweek", Equals, rows[builtinFuncNum-1][0].(string)) + const builtinFuncNum = 275 + require.Equal(t, len(rows), builtinFuncNum) + require.Equal(t, rows[0][0].(string), "abs") + require.Equal(t, rows[builtinFuncNum-1][0].(string), "yearweek") } -func (s *testSuite5) TestShowClusterConfig(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowClusterConfig(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") var confItems [][]types.Datum @@ -1396,7 +1421,7 @@ func (s *testSuite5) TestShowClusterConfig(c *C) { var confFunc executor.TestShowClusterConfigFunc = func() ([][]types.Datum, error) { return confItems, confErr } - tk.Se.SetValue(executor.TestShowClusterConfigKey, confFunc) + tk.Session().SetValue(executor.TestShowClusterConfigKey, confFunc) strs2Items := func(strs ...string) []types.Datum { items := make([]types.Datum, 0, len(strs)) for _, s := range strs { @@ -1418,40 +1443,42 @@ func (s *testSuite5) TestShowClusterConfig(c *C) { "tikv 127.0.0.1:3333 log.level info")) confErr = fmt.Errorf("something unknown error") - c.Assert(tk.QueryToErr("show config"), ErrorMatches, confErr.Error()) + require.EqualError(t, tk.QueryToErr("show config"), confErr.Error()) } -func (s *testSuite5) TestInvisibleCoprCacheConfig(c *C) { - se1, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - tk := testkit.NewTestKitWithSession(c, s.store, se1) +func TestInvisibleCoprCacheConfig(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) rows := tk.MustQuery("show variables like '%config%'").Rows() - c.Assert(len(rows), Equals, 1) + require.Equal(t, 1, len(rows)) configValue := rows[0][1].(string) coprCacheVal := "\t\t\"copr-cache\": {\n" + "\t\t\t\"capacity-mb\": 1000\n" + "\t\t},\n" - c.Assert(strings.Contains(configValue, coprCacheVal), Equals, true) + require.Equal(t, true, strings.Contains(configValue, coprCacheVal)) } -func (s *testSuite5) TestInvisibleGlobalKillConfig(c *C) { - se1, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - tk := testkit.NewTestKitWithSession(c, s.store, se1) +func TestInvisibleGlobalKillConfig(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) rows := tk.MustQuery("show variables like '%config%'").Rows() - c.Assert(len(rows), Equals, 1) + require.Equal(t, 1, len(rows)) configValue := rows[0][1].(string) globalKillVal := "global-kill" - c.Assert(strings.Contains(configValue, globalKillVal), Equals, false) + require.Equal(t, false, strings.Contains(configValue, globalKillVal)) } -func (s *testSerialSuite1) TestShowCreateTableWithIntegerDisplayLengthWarnings(c *C) { +func TestShowCreateTableWithIntegerDisplayLengthWarnings(t *testing.T) { parsertypes.TiDBStrictIntegerDisplayWidth = true defer func() { parsertypes.TiDBStrictIntegerDisplayWidth = false }() - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1510,8 +1537,10 @@ func (s *testSerialSuite1) TestShowCreateTableWithIntegerDisplayLengthWarnings(c ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) } -func (s *testSuite5) TestShowVar(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowVar(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) var showSQL string sessionVars := make([]string, 0, len(variable.GetSysVars())) globalVars := make([]string, 0, len(variable.GetSysVars())) @@ -1531,34 +1560,36 @@ func (s *testSuite5) TestShowVar(c *C) { sessionVarsStr := strings.Join(sessionVars, "','") showSQL = "show variables where variable_name in('" + sessionVarsStr + "')" res := tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, len(sessionVars)) + require.Len(t, res.Rows(), len(sessionVars)) showSQL = "show global variables where variable_name in('" + sessionVarsStr + "')" res = tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, 0) + require.Len(t, res.Rows(), 0) globalVarsStr := strings.Join(globalVars, "','") showSQL = "show variables where variable_name in('" + globalVarsStr + "')" res = tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, len(globalVars)) + require.Len(t, res.Rows(), len(globalVars)) showSQL = "show global variables where variable_name in('" + globalVarsStr + "')" res = tk.MustQuery(showSQL) - c.Check(res.Rows(), HasLen, len(globalVars)) + require.Len(t, res.Rows(), len(globalVars)) // Test a known hidden variable. res = tk.MustQuery("show variables like '" + variable.TiDBPartitionPruneMode + "'") - c.Check(res.Rows(), HasLen, 0) + require.Len(t, res.Rows(), 0) res = tk.MustQuery("show global variables like '" + variable.TiDBPartitionPruneMode + "'") - c.Check(res.Rows(), HasLen, 0) + require.Len(t, res.Rows(), 0) // Test Hidden tx_read_ts res = tk.MustQuery("show variables like '%tx_read_ts'") - c.Check(res.Rows(), HasLen, 0) + require.Len(t, res.Rows(), 0) // Test Hidden tidb_enable_streaming res = tk.MustQuery("show variables like '%tidb_enable_streaming%';") - c.Check(res.Rows(), HasLen, 0) + require.Len(t, res.Rows(), 0) } -func (s *testSuite5) TestIssue19507(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue19507(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE t2(a int primary key, b int unique, c int not null, unique index (c));") tk.MustQuery("SHOW INDEX IN t2;").Check( @@ -1579,8 +1610,10 @@ func (s *testSuite5) TestIssue19507(c *C) { } // TestShowPerformanceSchema tests for Issue 19231 -func (s *testSuite5) TestShowPerformanceSchema(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowPerformanceSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // Ideally we should create a new performance_schema table here with indices that we run the tests on. // However, its not possible to create a new performance_schema table since its a special in memory table. // Instead the test below uses the default index on the table. @@ -1589,21 +1622,25 @@ func (s *testSuite5) TestShowPerformanceSchema(c *C) { "events_statements_summary_by_digest 0 SCHEMA_NAME 2 DIGEST A 0 YES BTREE YES NO")) } -func (s *testSuite5) TestShowCreatePlacementPolicy(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCreatePlacementPolicy(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("CREATE PLACEMENT POLICY xyz PRIMARY_REGION='us-east-1' REGIONS='us-east-1,us-east-2' FOLLOWERS=4") tk.MustQuery("SHOW CREATE PLACEMENT POLICY xyz").Check(testkit.Rows("xyz CREATE PLACEMENT POLICY `xyz` PRIMARY_REGION=\"us-east-1\" REGIONS=\"us-east-1,us-east-2\" FOLLOWERS=4")) // non existent policy err := tk.QueryToErr("SHOW CREATE PLACEMENT POLICY doesnotexist") - c.Assert(err.Error(), Equals, infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs("doesnotexist").Error()) + require.Equal(t, infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs("doesnotexist").Error(), err.Error()) // alter and try second example tk.MustExec("ALTER PLACEMENT POLICY xyz FOLLOWERS=4") tk.MustQuery("SHOW CREATE PLACEMENT POLICY xyz").Check(testkit.Rows("xyz CREATE PLACEMENT POLICY `xyz` FOLLOWERS=4")) tk.MustExec("DROP PLACEMENT POLICY xyz") } -func (s *testSuite5) TestShowTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowTemporaryTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create global temporary table t1 (id int) on commit delete rows") tk.MustExec("create global temporary table t3 (i int primary key, j int) on commit delete rows") @@ -1662,8 +1699,10 @@ func (s *testSuite5) TestShowTemporaryTable(c *C) { tk.MustQuery("show create table t7").Check(testkit.Rows("t7 " + expect)) } -func (s *testSuite5) TestShowCachedTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowCachedTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t1 (id int)") tk.MustExec("alter table t1 cache") diff --git a/executor/simple.go b/executor/simple.go index 0de5688e4c0cb..ae3e0b197f8fc 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -967,25 +967,17 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt) if !ok { return errors.Trace(ErrPasswordFormat) } - stmt, err := exec.ParseWithParams(ctx, true, + _, _, err := exec.ExecRestrictedSQL(ctx, nil, `UPDATE %n.%n SET authentication_string=%?, plugin=%? WHERE Host=%? and User=%?;`, mysql.SystemDB, mysql.UserTable, pwd, spec.AuthOpt.AuthPlugin, strings.ToLower(spec.User.Hostname), spec.User.Username, ) - if err != nil { - return err - } - _, _, err = exec.ExecRestrictedStmt(ctx, stmt) if err != nil { failedUsers = append(failedUsers, spec.User.String()) } } if len(privData) > 0 { - stmt, err := exec.ParseWithParams(ctx, true, "INSERT INTO %n.%n (Host, User, Priv) VALUES (%?,%?,%?) ON DUPLICATE KEY UPDATE Priv = values(Priv)", mysql.SystemDB, mysql.GlobalPrivTable, spec.User.Hostname, spec.User.Username, string(hack.String(privData))) - if err != nil { - return err - } - _, _, err = exec.ExecRestrictedStmt(ctx, stmt) + _, _, err := exec.ExecRestrictedSQL(ctx, nil, "INSERT INTO %n.%n (Host, User, Priv) VALUES (%?,%?,%?) ON DUPLICATE KEY UPDATE Priv = values(Priv)", mysql.SystemDB, mysql.GlobalPrivTable, spec.User.Hostname, spec.User.Username, string(hack.String(privData))) if err != nil { failedUsers = append(failedUsers, spec.User.String()) } @@ -1359,11 +1351,7 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e func userExists(ctx context.Context, sctx sessionctx.Context, name string, host string) (bool, error) { exec := sctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, `SELECT * FROM %n.%n WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, name, strings.ToLower(host)) - if err != nil { - return false, err - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmt) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, `SELECT * FROM %n.%n WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, name, strings.ToLower(host)) if err != nil { return false, err } @@ -1442,11 +1430,7 @@ func (e *SimpleExec) executeSetPwd(ctx context.Context, s *ast.SetPwdStmt) error // update mysql.user exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(ctx, true, `UPDATE %n.%n SET authentication_string=%? WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, pwd, u, strings.ToLower(h)) - if err != nil { - return err - } - _, _, err = exec.ExecRestrictedStmt(ctx, stmt) + _, _, err = exec.ExecRestrictedSQL(ctx, nil, `UPDATE %n.%n SET authentication_string=%? WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, pwd, u, strings.ToLower(h)) if err != nil { return err } diff --git a/executor/simple_test.go b/executor/simple_test.go index 5271f206bc6fe..c97fea5d84fa5 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -38,25 +38,6 @@ import ( "github.com/pingcap/tidb/util/testutil" ) -func (s *testSuite3) TestCharsetDatabase(c *C) { - tk := testkit.NewTestKit(c, s.store) - testSQL := `create database if not exists cd_test_utf8 CHARACTER SET utf8 COLLATE utf8_bin;` - tk.MustExec(testSQL) - - testSQL = `create database if not exists cd_test_latin1 CHARACTER SET latin1 COLLATE latin1_swedish_ci;` - tk.MustExec(testSQL) - - testSQL = `use cd_test_utf8;` - tk.MustExec(testSQL) - tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows("utf8")) - tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("utf8_bin")) - - testSQL = `use cd_test_latin1;` - tk.MustExec(testSQL) - tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows("latin1")) - tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("latin1_swedish_ci")) -} - func (s *testSuite3) TestDo(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("do 1, @a:=1") diff --git a/executor/slow_query_test.go b/executor/slow_query_test.go index 9828263402ac1..7bce5d75fe547 100644 --- a/executor/slow_query_test.go +++ b/executor/slow_query_test.go @@ -18,12 +18,12 @@ import ( "bufio" "bytes" "context" + "fmt" "os" "strings" "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" - "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) func parseLog(retriever *slowQueryRetriever, sctx sessionctx.Context, reader *bufio.Reader, logNum int) ([][]types.Datum, error) { @@ -77,7 +77,7 @@ func parseSlowLog(sctx sessionctx.Context, reader *bufio.Reader, logNum int) ([] return rows, err } -func (s *testExecSerialSuite) TestParseSlowLogPanic(c *C) { +func TestParseSlowLogPanic(t *testing.T) { slowLogStr := `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 @@ -98,21 +98,21 @@ func (s *testExecSerialSuite) TestParseSlowLogPanic(c *C) { # Prev_stmt: update t set i = 1; use test; select * from t;` - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/errorMockParseSlowLogPanic", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/errorMockParseSlowLogPanic", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/errorMockParseSlowLogPanic"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/errorMockParseSlowLogPanic")) }() reader := bufio.NewReader(bytes.NewBufferString(slowLogStr)) loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) sctx := mock.NewContext() sctx.GetSessionVars().TimeZone = loc _, err = parseSlowLog(sctx, reader, 64) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "panic test") + require.Error(t, err) + require.Equal(t, err.Error(), "panic test") } -func (s *testExecSuite) TestParseSlowLogFile(c *C) { +func TestParseSlowLogFile(t *testing.T) { slowLogStr := `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 @@ -141,16 +141,16 @@ use test; select * from t;` reader := bufio.NewReader(bytes.NewBufferString(slowLogStr)) loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) ctx := mock.NewContext() ctx.GetSessionVars().TimeZone = loc rows, err := parseSlowLog(ctx, reader, 64) - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, 1) + require.NoError(t, err) + require.Len(t, rows, 1) recordString := "" for i, value := range rows[0] { str, err := value.ToString() - c.Assert(err, IsNil) + require.NoError(t, err) if i > 0 { recordString += "," } @@ -163,17 +163,17 @@ select * from t;` `Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` + `0,0,1,0,1,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + `update t set i = 1;,select * from t;` - c.Assert(expectRecordString, Equals, recordString) + require.Equal(t, expectRecordString, recordString) // Issue 20928 reader = bufio.NewReader(bytes.NewBufferString(slowLogStr)) rows, err = parseSlowLog(ctx, reader, 1) - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, 1) + require.NoError(t, err) + require.Len(t, rows, 1) recordString = "" for i, value := range rows[0] { str, err := value.ToString() - c.Assert(err, IsNil) + require.NoError(t, err) if i > 0 { recordString += "," } @@ -186,7 +186,7 @@ select * from t;` `Cop_backoff_regionMiss_total_times: 200 Cop_backoff_regionMiss_total_time: 0.2 Cop_backoff_regionMiss_max_time: 0.2 Cop_backoff_regionMiss_max_addr: 127.0.0.1 Cop_backoff_regionMiss_avg_time: 0.2 Cop_backoff_regionMiss_p90_time: 0.2 Cop_backoff_rpcPD_total_times: 200 Cop_backoff_rpcPD_total_time: 0.2 Cop_backoff_rpcPD_max_time: 0.2 Cop_backoff_rpcPD_max_addr: 127.0.0.1 Cop_backoff_rpcPD_avg_time: 0.2 Cop_backoff_rpcPD_p90_time: 0.2 Cop_backoff_rpcTiKV_total_times: 200 Cop_backoff_rpcTiKV_total_time: 0.2 Cop_backoff_rpcTiKV_max_time: 0.2 Cop_backoff_rpcTiKV_max_addr: 127.0.0.1 Cop_backoff_rpcTiKV_avg_time: 0.2 Cop_backoff_rpcTiKV_p90_time: 0.2,` + `0,0,1,0,1,1,,60e9378c746d9a2be1c791047e008967cf252eb6de9167ad3aa6098fa2d523f4,` + `update t set i = 1;,select * from t;` - c.Assert(expectRecordString, Equals, recordString) + require.Equal(t, expectRecordString, recordString) // fix sql contain '# ' bug slowLog := bytes.NewBufferString( @@ -204,7 +204,7 @@ select * from t; `) reader = bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader, 64) - c.Assert(err, IsNil) + require.NoError(t, err) // test for time format compatibility. slowLog = bytes.NewBufferString( @@ -215,14 +215,14 @@ select * from t; `) reader = bufio.NewReader(slowLog) rows, err = parseSlowLog(ctx, reader, 64) - c.Assert(err, IsNil) - c.Assert(len(rows) == 2, IsTrue) + require.NoError(t, err) + require.Len(t, rows, 2) t0Str, err := rows[0][0].ToString() - c.Assert(err, IsNil) - c.Assert(t0Str, Equals, "2019-04-28 15:24:04.309074") + require.NoError(t, err) + require.Equal(t, t0Str, "2019-04-28 15:24:04.309074") t1Str, err := rows[1][0].ToString() - c.Assert(err, IsNil) - c.Assert(t1Str, Equals, "2019-04-24 19:41:21.716221") + require.NoError(t, err) + require.Equal(t, t1Str, "2019-04-24 19:41:21.716221") // Add parse error check. slowLog = bytes.NewBufferString( @@ -232,16 +232,16 @@ select * from t; `) reader = bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader, 64) - c.Assert(err, IsNil) + require.NoError(t, err) warnings := ctx.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(warnings, HasLen, 1) - c.Assert(warnings[0].Err.Error(), Equals, "Parse slow log at line 2, failed field is Succ, failed value is abc, error is strconv.ParseBool: parsing \"abc\": invalid syntax") + require.Len(t, warnings, 1) + require.Equal(t, warnings[0].Err.Error(), "Parse slow log at line 2, failed field is Succ, failed value is abc, error is strconv.ParseBool: parsing \"abc\": invalid syntax") } // It changes variable.MaxOfMaxAllowedPacket, so must be stayed in SerialSuite. -func (s *testExecSerialSuite) TestParseSlowLogFileSerial(c *C) { +func TestParseSlowLogFileSerial(t *testing.T) { loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) ctx := mock.NewContext() ctx.GetSessionVars().TimeZone = loc // test for bufio.Scanner: token too long. @@ -256,33 +256,33 @@ select * from t; slowLog.WriteString(sql) reader := bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader, 64) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "single line length exceeds limit: 65536") + require.Error(t, err) + require.EqualError(t, err, "single line length exceeds limit: 65536") variable.MaxOfMaxAllowedPacket = originValue reader = bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, reader, 64) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testExecSuite) TestSlowLogParseTime(c *C) { +func TestSlowLogParseTime(t *testing.T) { t1Str := "2019-01-24T22:32:29.313255+08:00" t2Str := "2019-01-24T22:32:29.313255" t1, err := ParseTime(t1Str) - c.Assert(err, IsNil) + require.NoError(t, err) loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) t2, err := time.ParseInLocation("2006-01-02T15:04:05.999999999", t2Str, loc) - c.Assert(err, IsNil) - c.Assert(t1.Unix(), Equals, t2.Unix()) + require.NoError(t, err) + require.Equal(t, t1.Unix(), t2.Unix()) t1Format := t1.In(loc).Format(logutil.SlowLogTimeFormat) - c.Assert(t1Format, Equals, t1Str) + require.Equal(t, t1Format, t1Str) } // TestFixParseSlowLogFile bugfix // sql select * from INFORMATION_SCHEMA.SLOW_QUERY limit 1; // ERROR 1105 (HY000): string "2019-05-12-11:23:29.61474688" doesn't has a prefix that matches format "2006-01-02-15:04:05.999999999 -0700", err: parsing time "2019-05-12-11:23:29.61474688" as "2006-01-02-15:04:05.999999999 -0700": cannot parse "" as "-0700" -func (s *testExecSuite) TestFixParseSlowLogFile(c *C) { +func TestFixParseSlowLogFile(t *testing.T) { slowLog := bytes.NewBufferString( `# Time: 2019-05-12-11:23:29.614327491 +0800 # Txn_start_ts: 405888132465033227 @@ -309,11 +309,11 @@ select * from t select * from t;`) scanner := bufio.NewReader(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) ctx := mock.NewContext() ctx.GetSessionVars().TimeZone = loc _, err = parseSlowLog(ctx, scanner, 64) - c.Assert(err, IsNil) + require.NoError(t, err) // Test parser error. slowLog = bytes.NewBufferString( @@ -323,14 +323,13 @@ select * from t; `) scanner = bufio.NewReader(slowLog) _, err = parseSlowLog(ctx, scanner, 64) - c.Assert(err, IsNil) + require.NoError(t, err) warnings := ctx.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(warnings, HasLen, 1) - c.Assert(warnings[0].Err.Error(), Equals, "Parse slow log at line 2, failed field is Txn_start_ts, failed value is 405888132465033227#, error is strconv.ParseUint: parsing \"405888132465033227#\": invalid syntax") - + require.Len(t, warnings, 1) + require.Equal(t, warnings[0].Err.Error(), "Parse slow log at line 2, failed field is Txn_start_ts, failed value is 405888132465033227#, error is strconv.ParseUint: parsing \"405888132465033227#\": invalid syntax") } -func (s *testExecSuite) TestSlowQueryRetriever(c *C) { +func TestSlowQueryRetriever(t *testing.T) { logData0 := "" logData1 := ` # Time: 2020-02-15T18:00:01.000000+08:00 @@ -356,7 +355,7 @@ select 7;` fileName2 := "tidb-slow-2020-02-16T19-04-05.01.log" fileName3 := "tidb-slow.log" fileNames := []string{fileName0, fileName1, fileName2, fileName3} - prepareLogs(c, logData, fileNames) + prepareLogs(t, logData, fileNames) defer func() { removeFiles(fileNames) }() @@ -447,7 +446,7 @@ select 7;` } loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) sctx := mock.NewContext() sctx.GetSessionVars().TimeZone = loc sctx.GetSessionVars().SlowQueryFile = fileName3 @@ -455,33 +454,33 @@ select 7;` extractor := &plannercore.SlowQueryExtractor{Enable: len(cas.startTime) > 0 && len(cas.endTime) > 0} if extractor.Enable { startTime, err := ParseTime(cas.startTime) - c.Assert(err, IsNil) + require.NoError(t, err) endTime, err := ParseTime(cas.endTime) - c.Assert(err, IsNil) + require.NoError(t, err) extractor.TimeRanges = []*plannercore.TimeRange{{StartTime: startTime, EndTime: endTime}} } retriever, err := newSlowQueryRetriever() - c.Assert(err, IsNil) + require.NoError(t, err) retriever.extractor = extractor err = retriever.initialize(context.Background(), sctx) - c.Assert(err, IsNil) - comment := Commentf("case id: %v", i) - c.Assert(retriever.files, HasLen, len(cas.files), comment) + require.NoError(t, err) + comment := fmt.Sprintf("case id: %v", i) + require.Equal(t, len(retriever.files), len(cas.files), comment) if len(retriever.files) > 0 { reader := bufio.NewReader(retriever.files[0].file) rows, err := parseLog(retriever, sctx, reader, 64) - c.Assert(err, IsNil) - c.Assert(len(rows), Equals, len(cas.querys), comment) + require.NoError(t, err) + require.Equal(t, len(rows), len(cas.querys), comment) for i, row := range rows { - c.Assert(row[len(row)-1].GetString(), Equals, cas.querys[i], comment) + require.Equal(t, row[len(row)-1].GetString(), cas.querys[i], comment) } } for i, file := range retriever.files { - c.Assert(file.file.Name(), Equals, cas.files[i]) - c.Assert(file.file.Close(), IsNil) + require.Equal(t, file.file.Name(), cas.files[i]) + require.NoError(t, file.file.Close()) } - c.Assert(retriever.close(), IsNil) + require.NoError(t, retriever.close()) } } @@ -525,12 +524,12 @@ func TestSplitbyColon(t *testing.T) { } for _, c := range cases { resFields, resValues := splitByColon(c.line) - assert.Equal(t, c.fields, resFields) - assert.Equal(t, c.values, resValues) + require.Equal(t, c.fields, resFields) + require.Equal(t, c.values, resValues) } } -func (s *testExecSuite) TestBatchLogForReversedScan(c *C) { +func TestBatchLogForReversedScan(t *testing.T) { logData0 := "" logData1 := ` # Time: 2020-02-15T18:00:01.000000+08:00 @@ -560,7 +559,7 @@ select 9;` fileName3 := "tidb-slow-2020-02-17T19-04-05.01.log" fileName4 := "tidb-slow.log" fileNames := []string{fileName0, fileName1, fileName2, fileName3, fileName4} - prepareLogs(c, logData, fileNames) + prepareLogs(t, logData, fileNames) defer func() { removeFiles(fileNames) }() @@ -617,7 +616,7 @@ select 9;` } loc, err := time.LoadLocation("Asia/Shanghai") - c.Assert(err, IsNil) + require.NoError(t, err) sctx := mock.NewContext() sctx.GetSessionVars().TimeZone = loc sctx.GetSessionVars().SlowQueryFile = fileName3 @@ -625,41 +624,40 @@ select 9;` extractor := &plannercore.SlowQueryExtractor{Enable: len(cas.startTime) > 0 && len(cas.endTime) > 0, Desc: true} if extractor.Enable { startTime, err := ParseTime(cas.startTime) - c.Assert(err, IsNil) + require.NoError(t, err) endTime, err := ParseTime(cas.endTime) - c.Assert(err, IsNil) + require.NoError(t, err) extractor.TimeRanges = []*plannercore.TimeRange{{StartTime: startTime, EndTime: endTime}} } retriever, err := newSlowQueryRetriever() - c.Assert(err, IsNil) + require.NoError(t, err) retriever.extractor = extractor sctx.GetSessionVars().SlowQueryFile = fileName4 err = retriever.initialize(context.Background(), sctx) - c.Assert(err, IsNil) - comment := Commentf("case id: %v", i) - c.Assert(retriever.files, HasLen, len(cas.files), comment) + require.NoError(t, err) + comment := fmt.Sprintf("case id: %v", i) if len(retriever.files) > 0 { reader := bufio.NewReader(retriever.files[0].file) offset := &offset{length: 0, offset: 0} rows, err := retriever.getBatchLogForReversedScan(context.Background(), reader, offset, 3) - c.Assert(err, IsNil) + require.NoError(t, err) for _, row := range rows { for j, log := range row { - c.Assert(log, Equals, cas.logs[0][j], comment) + require.Equal(t, log, cas.logs[0][j], comment) } } } - c.Assert(retriever.close(), IsNil) + require.NoError(t, retriever.close()) } } -func prepareLogs(c *C, logData []string, fileNames []string) { +func prepareLogs(t *testing.T, logData []string, fileNames []string) { writeFile := func(file string, data string) { f, err := os.OpenFile(file, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = f.Write([]byte(data)) - c.Assert(f.Close(), IsNil) - c.Assert(err, IsNil) + require.NoError(t, err) + require.NoError(t, f.Close()) } for i, log := range logData { diff --git a/executor/sort_test.go b/executor/sort_test.go index aa87075115076..d895e93153a26 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -19,45 +19,50 @@ import ( "fmt" "os" "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSerialSuite1) TestSortInDisk(c *C) { - s.testSortInDisk(c, false) - s.testSortInDisk(c, true) +func TestSortInDisk(t *testing.T) { + testSortInDisk(t, false) + testSortInDisk(t, true) } -func (s *testSerialSuite1) testSortInDisk(c *C, removeDir bool) { - defer config.RestoreFunc()() +func testSortInDisk(t *testing.T, removeDir bool) { + restore := config.RestoreFunc() + defer restore() config.UpdateGlobal(func(conf *config.Config) { conf.OOMUseTmpStorage = true + conf.OOMAction = config.OOMActionLog + conf.TempStoragePath = t.TempDir() }) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)")) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill")) }() - - tk := testkit.NewTestKit(c, s.store) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") sm := &mockSessionManager1{ PS: make([]*util.ProcessInfo, 0), } - tk.Se.SetSessionManager(sm) - s.domain.ExpensiveQueryHandle().SetSessionManager(sm) + tk.Session().SetSessionManager(sm) + dom.ExpensiveQueryHandle().SetSessionManager(sm) if removeDir { - c.Assert(os.RemoveAll(config.GetGlobalConfig().TempStoragePath), IsNil) + require.Nil(t, os.RemoveAll(config.GetGlobalConfig().TempStoragePath)) defer func() { _, err := os.Stat(config.GetGlobalConfig().TempStoragePath) if err != nil { - c.Assert(os.IsExist(err), IsTrue) + require.True(t, os.IsExist(err)) } }() } @@ -79,32 +84,34 @@ func (s *testSerialSuite1) testSortInDisk(c *C, removeDir bool) { tk.MustExec(buf.String()) result := tk.MustQuery("select * from t order by c1") for i := 0; i < 1024; i++ { - c.Assert(result.Rows()[i][0].(string), Equals, fmt.Sprint(i)) - c.Assert(result.Rows()[i][1].(string), Equals, fmt.Sprint(i)) - c.Assert(result.Rows()[i][2].(string), Equals, fmt.Sprint(i)) + require.Equal(t, fmt.Sprint(i), result.Rows()[i][0].(string)) + require.Equal(t, fmt.Sprint(i), result.Rows()[i][1].(string)) + require.Equal(t, fmt.Sprint(i), result.Rows()[i][2].(string)) } - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.BytesConsumed(), Equals, int64(0)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) + require.Equal(t, int64(0), tk.Session().GetSessionVars().StmtCtx.MemTracker.BytesConsumed()) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), int64(0)) + require.Equal(t, int64(0), tk.Session().GetSessionVars().StmtCtx.DiskTracker.BytesConsumed()) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), int64(0)) } -func (s *testSerialSuite1) TestIssue16696(c *C) { +func TestIssue16696(t *testing.T) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.OOMUseTmpStorage = true + conf.OOMAction = config.OOMActionLog + conf.TempStoragePath = t.TempDir() }) alarmRatio := variable.MemoryUsageAlarmRatio.Load() variable.MemoryUsageAlarmRatio.Store(0.0) defer variable.MemoryUsageAlarmRatio.Store(alarmRatio) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)"), IsNil) - defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill"), IsNil) - }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testRowContainerSpill", "return(true)"), IsNil) - defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testRowContainerSpill"), IsNil) }() - tk := testkit.NewTestKit(c, s.store) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)")) + defer require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testRowContainerSpill", "return(true)")) + defer require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testRowContainerSpill")) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (`a` int(11) DEFAULT NULL,`b` int(11) DEFAULT NULL)") @@ -119,10 +126,10 @@ func (s *testSerialSuite1) TestIssue16696(c *C) { line := fmt.Sprintf("%v", row) disk := fmt.Sprintf("%v", row[length-1]) if strings.Contains(line, "Sort") || strings.Contains(line, "HashJoin") { - c.Assert(strings.Contains(disk, "0 Bytes"), IsFalse) - c.Assert(strings.Contains(disk, "MB") || + require.NotContains(t, disk, "0 Bytes") + require.True(t, strings.Contains(disk, "MB") || strings.Contains(disk, "KB") || - strings.Contains(disk, "Bytes"), IsTrue) + strings.Contains(disk, "Bytes")) } } } diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 03edfbb6b7ae1..8ce913abdee1b 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -16,19 +16,20 @@ package executor_test import ( "fmt" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { +func TestExactStalenessTransaction(t *testing.T) { testcases := []struct { name string preSQL string @@ -68,30 +69,33 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { zone: "", }, } - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") for _, testcase := range testcases { - c.Log(testcase.name) - failpoint.Enable("tikvclient/injectTxnScope", - fmt.Sprintf(`return("%v")`, testcase.zone)) + t.Log(testcase.name) + require.NoError(t, failpoint.Enable("tikvclient/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone))) tk.MustExec(testcase.preSQL) tk.MustExec(testcase.sql) - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) + require.Equal(t, testcase.IsStaleness, tk.Session().GetSessionVars().TxnCtx.IsStaleness) if testcase.expectPhysicalTS > 0 { - c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, testcase.expectPhysicalTS) + require.Equal(t, testcase.expectPhysicalTS, oracle.ExtractPhysical(tk.Session().GetSessionVars().TxnCtx.StartTS)) } else if !testcase.IsStaleness { curTS := oracle.ExtractPhysical(oracle.GoTimeToTS(time.Now())) - startTS := oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS) - c.Assert(curTS-startTS, Less, time.Second.Milliseconds()) - c.Assert(startTS-curTS, Less, time.Second.Milliseconds()) + startTS := oracle.ExtractPhysical(tk.Session().GetSessionVars().TxnCtx.StartTS) + require.Less(t, curTS-startTS, time.Second.Milliseconds()) + require.Less(t, startTS-curTS, time.Second.Milliseconds()) } tk.MustExec("commit") } - failpoint.Disable("tikvclient/injectTxnScope") + require.NoError(t, failpoint.Disable("tikvclient/injectTxnScope")) } -func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectAsOf(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" @@ -146,23 +150,23 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { } for _, testcase := range testcases1 { - c.Log(testcase.name) + t.Log(testcase.name) if len(testcase.setTxnSQL) > 0 { tk.MustExec(testcase.setTxnSQL) } - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, testcase.expectPhysicalTS)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, testcase.expectPhysicalTS))) rs, err := tk.Exec(testcase.sql) if len(testcase.errorStr) != 0 { - c.Assert(err, ErrorMatches, testcase.errorStr) + require.Regexp(t, testcase.errorStr, err.Error()) continue } - c.Assert(err, IsNil, Commentf("sql:%s, error stack %v", testcase.sql, errors.ErrorStack(err))) + require.NoErrorf(t, err, "sql:%s, error stack %v", testcase.sql, errors.ErrorStack(err)) if rs != nil { rs.Close() } - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO")) if len(testcase.setTxnSQL) > 0 { - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) } } @@ -222,29 +226,31 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { } for _, testcase := range testcases2 { - c.Log(testcase.name) + t.Log(testcase.name) if testcase.preSec > 0 { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix())), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-testcase.preSec)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix()))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-testcase.preSec))) } rs, err := tk.Exec(testcase.sql) if len(testcase.errorStr) != 0 { - c.Assert(err, ErrorMatches, testcase.errorStr) + require.Regexp(t, testcase.errorStr, err.Error()) continue } - c.Assert(err, IsNil, Commentf("sql:%s, error stack %v", testcase.sql, errors.ErrorStack(err))) + require.NoError(t, err, "sql:%s, error stack %v", testcase.sql, errors.ErrorStack(err)) if rs != nil { rs.Close() } if testcase.preSec > 0 { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/injectNow"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectNow")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO")) } } } -func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleReadKVRequest(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" @@ -290,25 +296,25 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { } tk.MustExec("set @@tidb_replica_read='closest-replicas'") for _, testcase := range testcases { - failpoint.Enable(testcase.assert, `return("sh")`) + require.NoError(t, failpoint.Enable(testcase.assert, `return("sh")`)) tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - failpoint.Disable(testcase.assert) + require.NoError(t, failpoint.Disable(testcase.assert)) } for _, testcase := range testcases { - failpoint.Enable(testcase.assert, `return("sh")`) + require.NoError(t, failpoint.Enable(testcase.assert, `return("sh")`)) tk.MustExec(`SET TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3)`) tk.MustExec(`begin;`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - failpoint.Disable(testcase.assert) + require.NoError(t, failpoint.Disable(testcase.assert)) } // assert follower read closest read for _, testcase := range testcases { - failpoint.Enable(testcase.assert, `return("sh")`) + require.NoError(t, failpoint.Enable(testcase.assert, `return("sh")`)) tk.MustQuery(testcase.sql) - failpoint.Disable(testcase.assert) + require.NoError(t, failpoint.Disable(testcase.assert)) } tk.MustExec(`insert into t1 (c,d,e) values (1,1,1);`) tk.MustExec(`insert into t1 (c,d,e) values (2,3,5);`) @@ -319,25 +325,27 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { tk.MustExec(`insert into t1 (c,d,e) values (5,0,5);`) // IndexLookUp Reader Executor rows1 := tk.MustQuery(fmt.Sprintf("select * from t1 AS OF TIMESTAMP '%v' use index (idx_d) where c < 5 and d < 5", tsv)).Rows() - c.Assert(rows1, HasLen, 2) + require.Len(t, rows1, 2) // IndexMerge Reader Executor rows2 := tk.MustQuery(fmt.Sprintf("select /*+ USE_INDEX_MERGE(t1, idx_d, idx_e) */ * from t1 AS OF TIMESTAMP '%v' where c <5 and (d =5 or e=5)", tsv)).Rows() - c.Assert(rows2, HasLen, 1) + require.Len(t, rows2, 1) // TableReader Executor rows3 := tk.MustQuery(fmt.Sprintf("select * from t1 AS OF TIMESTAMP '%v' where c < 6", tsv)).Rows() - c.Assert(rows3, HasLen, 2) + require.Len(t, rows3, 2) // IndexReader Executor rows4 := tk.MustQuery(fmt.Sprintf("select /*+ USE_INDEX(t1, idx_d) */ d from t1 AS OF TIMESTAMP '%v' where c < 5 and d < 1;", tsv)).Rows() - c.Assert(rows4, HasLen, 0) + require.Len(t, rows4, 0) // point get executor rows5 := tk.MustQuery(fmt.Sprintf("select * from t1 AS OF TIMESTAMP '%v' where c = 3;", tsv)).Rows() - c.Assert(rows5, HasLen, 0) + require.Len(t, rows5, 0) rows6 := tk.MustQuery(fmt.Sprintf("select * from t1 AS OF TIMESTAMP '%v' where c in (3,4,5);", tsv)).Rows() - c.Assert(rows6, HasLen, 0) + require.Len(t, rows6, 0) } -func (s *testStaleTxnSuite) TestStalenessAndHistoryRead(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStalenessAndHistoryRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" @@ -350,76 +358,78 @@ func (s *testStaleTxnSuite) TestStalenessAndHistoryRead(c *C) { time1 := time.Now() time1TS := oracle.GoTimeToTS(time1) - schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer1 := tk.Session().GetInfoSchema().SchemaMetaVersion() tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") tk.MustExec(`drop table if exists t`) time.Sleep(50 * time.Millisecond) time2 := time.Now() time2TS := oracle.GoTimeToTS(time2) - schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer2 := tk.Session().GetInfoSchema().SchemaMetaVersion() // test set txn as of will flush/mutex tidb_snapshot tk.MustExec(fmt.Sprintf(`set @@tidb_snapshot="%s"`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, time1TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, time1TS, tk.Session().GetSessionVars().SnapshotTS) + require.NotNil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, time2TS) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().SnapshotTS) + require.NotNil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, time2TS, tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) // test tidb_snapshot will flush/mutex set txn as of tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, time1TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, time1TS, tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) + require.NotNil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec(fmt.Sprintf(`set @@tidb_snapshot="%s"`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, time2TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) + require.Equal(t, time2TS, tk.Session().GetSessionVars().SnapshotTS) + require.NotNil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) // test start txn will flush/mutex tidb_snapshot tk.MustExec(fmt.Sprintf(`set @@tidb_snapshot="%s"`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, time1TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, time1TS, tk.Session().GetSessionVars().SnapshotTS) + require.NotNil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, time2TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().SnapshotTS) + require.Equal(t, time2TS, tk.Session().GetSessionVars().TxnCtx.StartTS) + require.Nil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec("commit") - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().SnapshotTS) + require.Nil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) // test snapshot mutex with txn tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, time2TS) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().SnapshotTS) + require.Equal(t, time2TS, tk.Session().GetSessionVars().TxnCtx.StartTS) + require.Nil(t, tk.Session().GetSessionVars().SnapshotInfoschema) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) err := tk.ExecToErr(`set @@tidb_snapshot="2020-10-08 16:45:26";`) - c.Assert(err, ErrorMatches, ".*Transaction characteristics can't be changed while a transaction is in progress") - c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) + require.Regexp(t, ".*Transaction characteristics can't be changed while a transaction is in progress", err.Error()) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().SnapshotTS) + require.Nil(t, tk.Session().GetSessionVars().SnapshotInfoschema) tk.MustExec("commit") // test set txn as of txn mutex with txn tk.MustExec("START TRANSACTION") - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) err = tk.ExecToErr("SET TRANSACTION READ ONLY AS OF TIMESTAMP '2020-10-08 16:46:26'") - c.Assert(err, ErrorMatches, ".*Transaction characteristics can't be changed while a transaction is in progress") - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Regexp(t, ".*Transaction characteristics can't be changed while a transaction is in progress", err.Error()) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) tk.MustExec("commit") } -func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTimeBoundedStalenessTxn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") @@ -463,57 +473,61 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { }, } for _, testcase := range testcases { - c.Log(testcase.name) - c.Assert(failpoint.Enable("tikvclient/injectSafeTS", - fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", - fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) + t.Log(testcase.name) + require.NoError(t, failpoint.Enable("tikvclient/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS))) tk.MustExec(testcase.sql) if testcase.compareWithSafeTS == 1 { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Greater, testcase.injectSafeTS) + require.Greater(t, tk.Session().GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS) } else if testcase.compareWithSafeTS == 0 { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) + require.Equal(t, testcase.injectSafeTS, tk.Session().GetSessionVars().TxnCtx.StartTS) } else { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Less, testcase.injectSafeTS) + require.Less(t, tk.Session().GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS) } tk.MustExec("commit") } - failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") - failpoint.Disable("tikvclient/injectSafeTS") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS")) + require.NoError(t, failpoint.Disable("tikvclient/injectSafeTS")) } -func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStalenessTransactionSchemaVer(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer1 := tk.Session().GetInfoSchema().SchemaMetaVersion() time1 := time.Now() tk.MustExec("alter table t add c int") // confirm schema changed - schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() - c.Assert(schemaVer1, Less, schemaVer2) + schemaVer2 := tk.Session().GetInfoSchema().SchemaMetaVersion() + require.Less(t, schemaVer1, schemaVer2) // get the specific old schema tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) // schema changed back to the newest tk.MustExec("commit") - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) // select does not affect the infoschema tk.MustExec(fmt.Sprintf(`SELECT * from t AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) } -func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { +func TestSetTransactionReadOnlyAsOf(t *testing.T) { t1, err := time.Parse(types.TimeFormat, "2016-09-21 09:53:04") - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, s.store) + require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -540,42 +554,42 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { } for _, testcase := range testcases { if testcase.injectSafeTS > 0 { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", - fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS))) } tk.MustExec(testcase.sql) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, testcase.expectedTS) + require.Equal(t, testcase.expectedTS, tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) tk.MustExec("begin") - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.expectedTS) + require.Equal(t, testcase.expectedTS, tk.Session().GetSessionVars().TxnCtx.StartTS) tk.MustExec("commit") - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) tk.MustExec("begin") - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Not(Equals), testcase.expectedTS) + require.NotEqual(t, tk.Session().GetSessionVars().TxnCtx.StartTS, testcase.expectedTS) tk.MustExec("commit") failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") } err = tk.ExecToErr(`SET TRANSACTION READ ONLY as of timestamp tidb_bounded_staleness(invalid1, invalid2')`) - c.Assert(err, NotNil) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Error(t, err) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) tk.MustExec(`SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`) err = tk.ExecToErr(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "start transaction read only as of is forbidden after set transaction read only as of") + require.Error(t, err) + require.Equal(t, "start transaction read only as of is forbidden after set transaction read only as of", err.Error()) tk.MustExec(`SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`) err = tk.ExecToErr(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "start transaction read only as of is forbidden after set transaction read only as of") + require.Error(t, err) + require.Equal(t, "start transaction read only as of is forbidden after set transaction read only as of", err.Error()) tk.MustExec("begin") - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(424394603102208000)) + require.Equal(t, uint64(424394603102208000), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) tk.MustExec("commit") tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`) } -func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { +func TestValidateReadOnlyInStalenessTransaction(t *testing.T) { errMsg1 := ".*only support read-only statement during read-only staleness transactions.*" errMsg2 := ".*select lock hasn't been supported in stale read yet.*" testcases := []struct { @@ -713,7 +727,9 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * errMsg: errMsg1, }, } - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -729,14 +745,14 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * tk.MustExec(`PREPARE stmt2 FROM 'select * from t';`) tk.MustExec(`set @@tidb_enable_noop_functions=1;`) for _, testcase := range testcases { - c.Log(testcase.name) + t.Log(testcase.name) tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) if testcase.isValidate { tk.MustExec(testcase.sql) } else { err := tk.ExecToErr(testcase.sql) - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, testcase.errMsg) + require.Error(t, err) + require.Regexp(t, testcase.errMsg, err.Error()) } tk.MustExec("commit") tk.MustExec("set transaction read only as of timestamp NOW(3);") @@ -744,16 +760,18 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * tk.MustExec(testcase.sql) } else { err := tk.ExecToErr(testcase.sql) - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, testcase.errMsg) + require.Error(t, err) + require.Regexp(t, testcase.errMsg, err.Error()) } // clean the status tk.MustExec("set transaction read only as of timestamp ''") } } -func (s *testStaleTxnSuite) TestSpecialSQLInStalenessTxn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSpecialSQLInStalenessTxn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testcases := []struct { name string @@ -798,16 +816,17 @@ func (s *testStaleTxnSuite) TestSpecialSQLInStalenessTxn(c *C) { } tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") for _, testcase := range testcases { - comment := Commentf(testcase.name) tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) + require.Equal(t, true, tk.Session().GetSessionVars().TxnCtx.IsStaleness, testcase.name) tk.MustExec(testcase.sql) - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) + require.Equal(t, testcase.sameSession, tk.Session().GetSessionVars().TxnCtx.IsStaleness, testcase.name) } } -func (s *testStaleTxnSuite) TestAsOfTimestampCompatibility(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAsOfTimestampCompatibility(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -852,7 +871,7 @@ func (s *testStaleTxnSuite) TestAsOfTimestampCompatibility(c *C) { for _, testcase := range testcases { tk.MustExec(testcase.beginSQL) err := tk.ExecToErr(testcase.sql) - c.Assert(err, ErrorMatches, ".*as of timestamp can't be set in transaction.*|.*Transaction characteristics can't be changed while a transaction is in progress") + require.Regexp(t, ".*as of timestamp can't be set in transaction.*|.*Transaction characteristics can't be changed while a transaction is in progress", err.Error()) tk.MustExec("commit") } tk.MustExec(`create table test.table1 (id int primary key, a int);`) @@ -861,8 +880,10 @@ func (s *testStaleTxnSuite) TestAsOfTimestampCompatibility(c *C) { tk.MustExec(fmt.Sprintf("explain analyze select * from test.table1 as of timestamp '%s' where id = 1;", time1.Format("2006-1-2 15:04:05.000"))) } -func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetTransactionInfoSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -876,32 +897,34 @@ func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer1 := tk.Session().GetInfoSchema().SchemaMetaVersion() time1 := time.Now() tk.MustExec("alter table t add c int") // confirm schema changed - schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer2 := tk.Session().GetInfoSchema().SchemaMetaVersion() time2 := time.Now() - c.Assert(schemaVer1, Less, schemaVer2) + require.Less(t, schemaVer1, schemaVer2) tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec("select * from t;") tk.MustExec("alter table t add d int") - schemaVer3 := tk.Se.GetInfoSchema().SchemaMetaVersion() + schemaVer3 := tk.Session().GetInfoSchema().SchemaMetaVersion() tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) tk.MustExec("begin;") - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + require.Equal(t, schemaVer1, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec("commit") tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) tk.MustExec("begin;") - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + require.Equal(t, schemaVer2, tk.Session().GetInfoSchema().SchemaMetaVersion()) tk.MustExec("commit") - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer3) + require.Equal(t, schemaVer3, tk.Session().GetInfoSchema().SchemaMetaVersion()) } -func (s *testStaleTxnSerialSuite) TestStaleSelect(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleSelect(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -928,7 +951,7 @@ func (s *testStaleTxnSerialSuite) TestStaleSelect(c *C) { // test stale select in txn tk.MustExec("begin") - c.Assert(tk.ExecToErr(staleSQL), NotNil) + require.NotNil(t, tk.ExecToErr(staleSQL)) tk.MustExec("commit") // test prepared stale select @@ -937,12 +960,12 @@ func (s *testStaleTxnSerialSuite) TestStaleSelect(c *C) { // test prepared stale select in txn tk.MustExec("begin") - c.Assert(tk.ExecToErr(staleSQL), NotNil) + require.NotNil(t, tk.ExecToErr(staleSQL)) tk.MustExec("commit") // test stale select in stale txn tk.MustExec(fmt.Sprintf(`start transaction read only as of timestamp '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.ExecToErr(staleSQL), NotNil) + require.NotNil(t, tk.ExecToErr(staleSQL)) tk.MustExec("commit") // test prepared stale select with schema change @@ -961,29 +984,33 @@ func (s *testStaleTxnSerialSuite) TestStaleSelect(c *C) { tk.MustQuery(fmt.Sprintf("select * from t as of timestamp '%s' where c=5", time6.Format("2006-1-2 15:04:05.000"))).Check(testkit.Rows("4 5 ")) } -func (s *testStaleTxnSuite) TestStaleReadFutureTime(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleReadFutureTime(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int)") // Setting tx_read_ts to a time in the future will fail. (One day before the 2038 problem) _, err := tk.Exec("start transaction read only as of timestamp '2038-01-18 03:14:07'") - c.Assert(err, ErrorMatches, "cannot set read timestamp to a future time") + require.Regexp(t, "cannot set read timestamp to a future time", err.Error()) // Transaction should not be started and read ts should not be set if check fails - c.Assert(tk.Se.GetSessionVars().InTxn(), IsFalse) - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.False(t, tk.Session().GetSessionVars().InTxn()) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) _, err = tk.Exec("set transaction read only as of timestamp '2038-01-18 03:14:07'") - c.Assert(err, ErrorMatches, "cannot set read timestamp to a future time") - c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + require.Regexp(t, "cannot set read timestamp to a future time", err.Error()) + require.Equal(t, uint64(0), tk.Session().GetSessionVars().TxnReadTS.PeakTxnReadTS()) _, err = tk.Exec("select * from t as of timestamp '2038-01-18 03:14:07'") - c.Assert(err, ErrorMatches, "cannot set read timestamp to a future time") + require.Regexp(t, "cannot set read timestamp to a future time", err.Error()) } -func (s *testStaleTxnSerialSuite) TestStaleReadPrepare(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleReadPrepare(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -1016,16 +1043,20 @@ func (s *testStaleTxnSerialSuite) TestStaleReadPrepare(c *C) { // test prepared stale select in stale txn tk.MustExec(fmt.Sprintf(`start transaction read only as of timestamp '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert("execute p1", NotNil) + _, err := tk.Exec("execute p1") + require.Error(t, err) tk.MustExec("commit") // assert execute prepared statement should be error after set transaction read only as of tk.MustExec(fmt.Sprintf(`set transaction read only as of timestamp '%s'`, time1.Format("2006-1-2 15:04:05.000"))) - c.Assert("execute p1", NotNil) + _, err = tk.Exec("execute p1") + require.Error(t, err) } -func (s *testStaleTxnSerialSuite) TestStmtCtxStaleFlag(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStmtCtxStaleFlag(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -1111,14 +1142,16 @@ func (s *testStaleTxnSerialSuite) TestStmtCtxStaleFlag(c *C) { failpoint.Enable("github.com/pingcap/tidb/exector/assertStmtCtxIsStaleness", fmt.Sprintf("return(%v)", testcase.hasStaleFlag)) tk.MustExec(testcase.sql) - failpoint.Disable("github.com/pingcap/tidb/exector/assertStmtCtxIsStaleness") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/exector/assertStmtCtxIsStaleness")) // assert stale read flag should be false after each statement execution - c.Assert(tk.Se.GetSessionVars().StmtCtx.IsStaleness, IsFalse) + require.False(t, tk.Session().GetSessionVars().StmtCtx.IsStaleness) } } -func (s *testStaleTxnSerialSuite) TestStaleSessionQuery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleSessionQuery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -1134,29 +1167,31 @@ func (s *testStaleTxnSerialSuite) TestStaleSessionQuery(c *C) { now := time.Now() tk.MustExec(`set @@tidb_read_staleness="-1"`) // query will use stale read - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix())), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-1)), IsNil) - c.Assert(tk.MustQuery("select * from t10;").Rows(), HasLen, 1) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/injectNow"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix()))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-1))) + require.Len(t, tk.MustQuery("select * from t10;").Rows(), 1) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectNow")) // begin transaction won't be affected by read staleness tk.MustExec("begin") tk.MustExec("insert into t10(id) values (2);") tk.MustExec("commit") tk.MustExec("insert into t10(id) values (3);") // query will still use staleness read - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix())), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-1)), IsNil) - c.Assert(tk.MustQuery("select * from t10").Rows(), HasLen, 1) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/injectNow"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, now.Unix()))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, now.Unix()-1))) + require.Len(t, tk.MustQuery("select * from t10").Rows(), 1) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectNow")) // assert stale read is not exist after empty the variable tk.MustExec(`set @@tidb_read_staleness=""`) - c.Assert(tk.MustQuery("select * from t10").Rows(), HasLen, 3) + require.Len(t, tk.MustQuery("select * from t10").Rows(), 3) } -func (s *testStaleTxnSerialSuite) TestStaleReadCompatibility(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleReadCompatibility(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer tk.MustExec("drop table if exists t") @@ -1171,36 +1206,38 @@ func (s *testStaleTxnSerialSuite) TestStaleReadCompatibility(c *C) { // assert select as of timestamp won't work after set transaction read only as of timestamp tk.MustExec(fmt.Sprintf("set transaction read only as of timestamp '%s';", t1.Format("2006-1-2 15:04:05"))) err := tk.ExecToErr(fmt.Sprintf("select * from t as of timestamp '%s';", t1.Format("2006-1-2 15:04:05"))) - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, ".*invalid as of timestamp: can't use select as of while already set transaction as of.*") + require.Error(t, err) + require.Regexp(t, ".*invalid as of timestamp: can't use select as of while already set transaction as of.*", err.Error()) // assert set transaction read only as of timestamp is consumed - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 3) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 3) // enable tidb_read_staleness tk.MustExec("set @@tidb_read_staleness='-1'") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, t1.Unix())), IsNil) - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 1) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/expression/injectNow", fmt.Sprintf(`return(%d)`, t1.Unix()))) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 1) // assert select as of timestamp during tidb_read_staleness - c.Assert(tk.MustQuery(fmt.Sprintf("select * from t as of timestamp '%s'", t2.Format("2006-1-2 15:04:05"))).Rows(), HasLen, 2) + require.Len(t, tk.MustQuery(fmt.Sprintf("select * from t as of timestamp '%s'", t2.Format("2006-1-2 15:04:05"))).Rows(), 2) // assert set transaction as of timestamp during tidb_read_staleness tk.MustExec(fmt.Sprintf("set transaction read only as of timestamp '%s';", t2.Format("2006-1-2 15:04:05"))) - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 2) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 2) // assert begin stale transaction during tidb_read_staleness tk.MustExec(fmt.Sprintf("start transaction read only as of timestamp '%v'", t2.Format("2006-1-2 15:04:05"))) - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 2) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 2) tk.MustExec("commit") // assert session query still is affected by tidb_read_staleness - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 1) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 1) // disable tidb_read_staleness tk.MustExec("set @@tidb_read_staleness=''") - c.Assert(tk.MustQuery("select * from t;").Rows(), HasLen, 3) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/injectNow"), IsNil) + require.Len(t, tk.MustQuery("select * from t;").Rows(), 3) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/expression/injectNow")) } -func (s *testStaleTxnSerialSuite) TestStaleReadNoExtraTSORequest(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStaleReadNoExtraTSORequest(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" safePointValue := "20160102-15:04:05 -0700" @@ -1214,34 +1251,34 @@ func (s *testStaleTxnSerialSuite) TestStaleReadNoExtraTSORequest(c *C) { tk.MustExec("create table t (id int);") time.Sleep(3 * time.Second) // statement stale read - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`)) tk.MustQuery("select * from t as of timestamp NOW() - INTERVAL 2 SECOND") - failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest")) // set and statement stale read tk.MustExec("set transaction read only as of timestamp NOW() - INTERVAL 2 SECOND") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`)) tk.MustQuery("select * from t") - failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest")) // stale read transaction - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`)) tk.MustExec("start transaction read only as of timestamp NOW() - INTERVAL 2 SECOND") tk.MustQuery("select * from t") tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest")) // set and stale read transaction tk.MustExec("set transaction read only as of timestamp NOW() - INTERVAL 2 SECOND") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`)) tk.MustExec("begin") tk.MustQuery("select * from t") tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest")) // use tidb_read_staleness tk.MustExec(`set @@tidb_read_staleness='-1'`) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/assertTSONotRequest", `return(true)`)) tk.MustQuery("select * from t") - failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/assertTSONotRequest")) } diff --git a/executor/statement_context_test.go b/executor/statement_context_test.go index 247063acf71de..32720bd6cc36b 100644 --- a/executor/statement_context_test.go +++ b/executor/statement_context_test.go @@ -16,13 +16,14 @@ package executor_test import ( "fmt" + "testing" "unicode/utf8" - . "github.com/pingcap/check" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) const ( @@ -30,8 +31,10 @@ const ( nonStrictModeSQL = "set sql_mode = ''" ) -func (s *testSuite1) TestStatementContext(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStatementContext(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table sc (a int)") tk.MustExec("insert sc values (1), (2)") @@ -53,9 +56,9 @@ func (s *testSuite1) TestStatementContext(c *C) { // UPDATE and DELETE do select request first which is handled by coprocessor. // In strict mode we expect error. _, err := tk.Exec("update sc set a = 4 where a > '1x'") - c.Assert(err, NotNil) + require.Error(t, err) _, err = tk.Exec("delete from sc where a < '1x'") - c.Assert(err, NotNil) + require.Error(t, err) tk.MustQuery("select * from sc where a > '1x'").Check(testkit.Rows("3")) // Non-strict mode never returns error. @@ -68,16 +71,16 @@ func (s *testSuite1) TestStatementContext(c *C) { tk.MustExec("create table sc2 (a varchar(255))") // Insert an invalid UTF8 tk.MustExec("insert sc2 values (unhex('4040ffff'))") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Greater, uint16(0)) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) tk.MustQuery("select * from sc2").Check(testkit.Rows("@@")) tk.MustExec(strictModeSQL) _, err = tk.Exec("insert sc2 values (unhex('4040ffff'))") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), "err %v", err) tk.MustExec("set @@tidb_skip_utf8_check = '1'") _, err = tk.Exec("insert sc2 values (unhex('4040ffff'))") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery("select length(a) from sc2").Check(testkit.Rows("2", "4")) tk.MustExec("set @@tidb_skip_utf8_check = '0'") @@ -89,17 +92,17 @@ func (s *testSuite1) TestStatementContext(c *C) { tk.MustExec(nonStrictModeSQL) tk.MustExec("insert sc3 values (unhex('4040ffff'))") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Greater, uint16(0)) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) tk.MustQuery("select * from sc3").Check(testkit.Rows("@@")) tk.MustExec(strictModeSQL) _, err = tk.Exec("insert sc3 values (unhex('4040ffff'))") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), "err %v", err) tk.MustExec("set @@tidb_skip_ascii_check = '1'") _, err = tk.Exec("insert sc3 values (unhex('4040ffff'))") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery("select length(a) from sc3").Check(testkit.Rows("2", "4")) // no placeholder in ASCII, so just insert '@@'... @@ -112,19 +115,19 @@ func (s *testSuite1) TestStatementContext(c *C) { tk.MustExec("create table t1(a varchar(100) charset utf8);") defer tk.MustExec("drop table if exists t1") tk.MustExec("insert t1 values (unhex('f09f8c80'))") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Greater, uint16(0)) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) tk.MustQuery("select * from t1").Check(testkit.Rows("")) tk.MustExec("insert t1 values (unhex('4040f09f8c80'))") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Greater, uint16(0)) + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(0)) tk.MustQuery("select * from t1").Check(testkit.Rows("", "@@")) tk.MustQuery("select length(a) from t1").Check(testkit.Rows("0", "2")) tk.MustExec(strictModeSQL) _, err = tk.Exec("insert t1 values (unhex('f09f8c80'))") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), "err %v", err) _, err = tk.Exec("insert t1 values (unhex('F0A48BAE'))") - c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), IsTrue, Commentf("err %v", err)) + require.Error(t, err) + require.Truef(t, terror.ErrorEqual(err, table.ErrTruncatedWrongValueForField), "err %v", err) config.UpdateGlobal(func(conf *config.Config) { conf.CheckMb4ValueInUTF8.Store(false) }) @@ -133,5 +136,5 @@ func (s *testSuite1) TestStatementContext(c *C) { conf.CheckMb4ValueInUTF8.Store(true) }) _, err = tk.Exec("insert t1 values (unhex('F0A48BAE'))") - c.Assert(err, NotNil) + require.Error(t, err) } diff --git a/executor/table_reader.go b/executor/table_reader.go index 0e2c66ba637b7..4b7678ebd961f 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -111,6 +111,10 @@ type TableReaderExecutor struct { // extraPIDColumnIndex is used for partition reader to add an extra partition ID column. extraPIDColumnIndex offsetOptional + + // If dummy flag is set, this is not a real TableReader, it just provides the KV ranges for UnionScan. + // Used by the temporary table, cached table. + dummy bool } // offsetOptional may be a positive integer, or invalid. @@ -128,6 +132,11 @@ func (i offsetOptional) value() int { return int(i - 1) } +// Table implements the dataSourceExecutor interface. +func (e *TableReaderExecutor) Table() table.Table { + return e.table +} + // Open initializes necessary variables for using this executor. func (e *TableReaderExecutor) Open(ctx context.Context) error { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { @@ -180,7 +189,7 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // Treat temporary table as dummy table, avoid sending distsql request to TiKV. // Calculate the kv ranges here, UnionScan rely on this kv ranges. // cached table and temporary table are similar - if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { kvReq, err := e.buildKVReq(ctx, firstPartRanges) if err != nil { return err @@ -218,7 +227,7 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error { - if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { // Treat temporary table as dummy table, avoid sending distsql request to TiKV. req.Reset() return nil @@ -263,7 +272,7 @@ func fillExtraPIDColumn(req *chunk.Chunk, extraPIDColumnIndex int, physicalID in // Close implements the Executor Close interface. func (e *TableReaderExecutor) Close() error { - if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + if e.dummy { return nil } @@ -321,9 +330,6 @@ func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges [ return nil, err } var builder distsql.RequestBuilder - if e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - builder.SetIsolationLevel(kv.RC) - } reqBuilder := builder.SetKeyRanges(kvRange) kvReq, err := reqBuilder. SetDAGRequest(e.dagPB). @@ -357,9 +363,6 @@ func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.R } else { reqBuilder = builder.SetHandleRanges(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) } - if e.ctx.GetSessionVars().StmtCtx.WeakConsistency { - reqBuilder.SetIsolationLevel(kv.RC) - } reqBuilder. SetDAGRequest(e.dagPB). SetStartTS(e.startTS). diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index 69ed5fc7fab7f..73044de929a8e 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -18,9 +18,9 @@ import ( "context" "fmt" "math/rand" + "testing" "github.com/cznic/mathutil" - . "github.com/pingcap/check" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" ) type requiredRowsSelectResult struct { @@ -153,7 +154,7 @@ func buildMockBaseExec(sctx sessionctx.Context) baseExecutor { return baseExec } -func (s *testExecSuite) TestTableReaderRequiredRows(c *C) { +func TestTableReaderRequiredRows(t *testing.T) { maxChunkSize := defaultCtx().GetSessionVars().MaxChunkSize testCases := []struct { totalRows int @@ -184,14 +185,14 @@ func (s *testExecSuite) TestTableReaderRequiredRows(c *C) { sctx := defaultCtx() ctx := mockDistsqlSelectCtxSet(testCase.totalRows, testCase.expectedRowsDS) exec := buildTableReader(sctx) - c.Assert(exec.Open(ctx), IsNil) + require.NoError(t, exec.Open(ctx)) chk := newFirstChunk(exec) for i := range testCase.requiredRows { chk.SetRequiredRows(testCase.requiredRows[i], maxChunkSize) - c.Assert(exec.Next(ctx, chk), IsNil) - c.Assert(chk.NumRows(), Equals, testCase.expectedRows[i]) + require.NoError(t, exec.Next(ctx, chk)) + require.Equal(t, testCase.expectedRows[i], chk.NumRows()) } - c.Assert(exec.Close(), IsNil) + require.NoError(t, exec.Close()) } } @@ -205,7 +206,7 @@ func buildIndexReader(sctx sessionctx.Context) Executor { return e } -func (s *testExecSuite) TestIndexReaderRequiredRows(c *C) { +func TestIndexReaderRequiredRows(t *testing.T) { maxChunkSize := defaultCtx().GetSessionVars().MaxChunkSize testCases := []struct { totalRows int @@ -236,13 +237,13 @@ func (s *testExecSuite) TestIndexReaderRequiredRows(c *C) { sctx := defaultCtx() ctx := mockDistsqlSelectCtxSet(testCase.totalRows, testCase.expectedRowsDS) exec := buildIndexReader(sctx) - c.Assert(exec.Open(ctx), IsNil) + require.NoError(t, exec.Open(ctx)) chk := newFirstChunk(exec) for i := range testCase.requiredRows { chk.SetRequiredRows(testCase.requiredRows[i], maxChunkSize) - c.Assert(exec.Next(ctx, chk), IsNil) - c.Assert(chk.NumRows(), Equals, testCase.expectedRows[i]) + require.NoError(t, exec.Next(ctx, chk)) + require.Equal(t, testCase.expectedRows[i], chk.NumRows()) } - c.Assert(exec.Close(), IsNil) + require.NoError(t, exec.Close()) } } diff --git a/executor/trace_test.go b/executor/trace_test.go index 0771151c63e2a..f8e8e91ddebd7 100644 --- a/executor/trace_test.go +++ b/executor/trace_test.go @@ -15,18 +15,22 @@ package executor_test import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/util/testkit" + "testing" + + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSuite1) TestTraceExec(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTraceExec(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") testSQL := `create table trace (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` tk.MustExec(testSQL) tk.MustExec("trace insert into trace (c1, c2, c3) values (1, 2, 3)") rows := tk.MustQuery("trace select * from trace where id = 0;").Rows() - c.Assert(len(rows), GreaterEqual, 1) + require.GreaterOrEqual(t, len(rows), 1) // +---------------------------+-----------------+------------+ // | operation | startTS | duration | @@ -40,16 +44,16 @@ func (s *testSuite1) TestTraceExec(c *C) { // | └─recordSet.Next | 22:08:38.249340 | 155.317µs | // +---------------------------+-----------------+------------+ rows = tk.MustQuery("trace format='row' select * from trace where id = 0;").Rows() - c.Assert(len(rows) > 1, IsTrue) - c.Assert(rowsOrdered(rows), IsTrue) + require.Greater(t, len(rows), 1) + require.True(t, rowsOrdered(rows)) rows = tk.MustQuery("trace format='row' delete from trace where id = 0").Rows() - c.Assert(len(rows) > 1, IsTrue) - c.Assert(rowsOrdered(rows), IsTrue) + require.Greater(t, len(rows), 1) + require.True(t, rowsOrdered(rows)) tk.MustExec("trace format='log' insert into trace (c1, c2, c3) values (1, 2, 3)") rows = tk.MustQuery("trace format='log' select * from trace where id = 0;").Rows() - c.Assert(len(rows), GreaterEqual, 1) + require.GreaterOrEqual(t, len(rows), 1) } func rowsOrdered(rows [][]interface{}) bool { @@ -67,12 +71,14 @@ func rowsOrdered(rows [][]interface{}) bool { return true } -func (s *testSuite1) TestTracePlanStmt(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTracePlanStmt(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table tp123(id int);") rows := tk.MustQuery("trace plan select * from tp123").Rows() - c.Assert(rows, HasLen, 1) - c.Assert(rows[0], HasLen, 1) - c.Assert(rows[0][0].(string), Matches, ".*zip") + require.Len(t, rows, 1) + require.Len(t, rows[0], 1) + require.Regexp(t, ".*zip", rows[0][0]) } diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 6133010ec29da..bdf521427dcda 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -18,7 +18,9 @@ import ( "fmt" "testing" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/benchdaily" "github.com/stretchr/testify/require" ) @@ -416,3 +418,36 @@ func TestForApplyAndUnionScan(t *testing.T) { tk.MustQuery("select c_int, c_str from t where (select count(*) from t1 where t1.c_int in (t.c_int, t.c_int + 2, t.c_int + 10)) > 2").Check(testkit.Rows()) tk.MustExec("rollback") } + +func BenchmarkUnionScanRead(b *testing.B) { + store, clean := testkit.CreateMockStore(b) + defer clean() + + tk := testkit.NewTestKit(b, store) + tk.MustExec("use test") + tk.MustExec(`create table t_us ( +c1 varchar(10), +c2 varchar(30), +c3 varchar(1), +c4 varchar(12), +c5 varchar(10), +c6 datetime);`) + tk.MustExec(`begin;`) + for i := 0; i < 8000; i++ { + tk.MustExec("insert into t_us values ('54321', '1234', '1', '000000', '7518', '2014-05-08')") + } + + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + tk.MustQuery("select * from t_us where c1 = '12345'").Check(testkit.Rows()) + } + b.StopTimer() +} + +func TestBenchDaily(t *testing.T) { + benchdaily.Run( + executor.BenchmarkReadLastLinesOfHugeLine, + BenchmarkUnionScanRead, + ) +} diff --git a/executor/update_test.go b/executor/update_test.go index 12211cc33fd66..03217a25ebf39 100644 --- a/executor/update_test.go +++ b/executor/update_test.go @@ -15,80 +15,35 @@ package executor_test import ( - "flag" - "fmt" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" - "github.com/tikv/client-go/v2/testutils" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -type testUpdateSuite struct { - cluster testutils.Cluster - store kv.Storage - domain *domain.Domain - *parser.Parser -} - -func (s *testUpdateSuite) SetUpSuite(c *C) { - s.Parser = parser.New() - flag.Lookup("mockTikv") - useMockTikv := *mockTikv - if useMockTikv { - store, err := mockstore.NewMockStore( - mockstore.WithClusterInspector(func(c testutils.Cluster) { - mockstore.BootstrapWithSingleStore(c) - s.cluster = c - }), - ) - c.Assert(err, IsNil) - s.store = store - session.SetSchemaLease(0) - session.DisableStats4Test() - } - d, err := session.BootstrapSession(s.store) - c.Assert(err, IsNil) - d.SetStatsUpdating(true) - s.domain = d -} - -func (s *testUpdateSuite) TearDownSuite(c *C) { - s.domain.Close() - c.Assert(s.store.Close(), IsNil) -} - -func (s *testUpdateSuite) TearDownTest(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - -func (s *testUpdateSuite) TestUpdateGenColInTxn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateGenColInTxn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table t(a bigint, b bigint as (a+1));`) tk.MustExec(`begin;`) tk.MustExec(`insert into t(a) values(1);`) err := tk.ExecToErr(`update t set b=6 where b=2;`) - c.Assert(err.Error(), Equals, "[planner:3105]The value specified for generated column 'b' in table 't' is not allowed.") + require.Equal(t, "[planner:3105]The value specified for generated column 'b' in table 't' is not allowed.", err.Error()) tk.MustExec(`commit;`) tk.MustQuery(`select * from t;`).Check(testkit.Rows( `1 2`)) } -func (s *testUpdateSuite) TestUpdateWithAutoidSchema(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateWithAutoidSchema(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`create table t1(id int primary key auto_increment, n int);`) tk.MustExec(`create table t2(id int primary key, n float auto_increment, key I_n(n));`) @@ -212,21 +167,25 @@ func (s *testUpdateSuite) TestUpdateWithAutoidSchema(c *C) { } } -func (s *testUpdateSuite) TestUpdateSchemaChange(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateSchemaChange(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table t(a bigint, b bigint as (a+1));`) tk.MustExec(`begin;`) tk.MustExec(`insert into t(a) values(1);`) err := tk.ExecToErr(`update t set b=6 where b=2;`) - c.Assert(err.Error(), Equals, "[planner:3105]The value specified for generated column 'b' in table 't' is not allowed.") + require.Equal(t, "[planner:3105]The value specified for generated column 'b' in table 't' is not allowed.", err.Error()) tk.MustExec(`commit;`) tk.MustQuery(`select * from t;`).Check(testkit.Rows( `1 2`)) } -func (s *testUpdateSuite) TestUpdateMultiDatabaseTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateMultiDatabaseTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop database if exists test2") tk.MustExec("create database test2") @@ -235,8 +194,10 @@ func (s *testUpdateSuite) TestUpdateMultiDatabaseTable(c *C) { tk.MustExec("update t, test2.t set test.t.a=1") } -func (s *testUpdateSuite) TestUpdateSwapColumnValues(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestUpdateSwapColumnValues(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1 (c_str varchar(40))") @@ -271,8 +232,10 @@ func (s *testUpdateSuite) TestUpdateSwapColumnValues(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("10 30 -10 -30", "20 30 -20 -30")) } -func (s *testUpdateSuite) TestMultiUpdateOnSameTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiUpdateOnSameTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(x int, y int)") @@ -328,16 +291,13 @@ func (s *testUpdateSuite) TestMultiUpdateOnSameTable(c *C) { `[planner:1706]Primary key/partition key update is not allowed since the table is updated both as 'm' and 'n'.`) } -var _ = SerialSuites(&testSuite11{&baseTestSuite{}}) +func TestUpdateClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() -type testSuite11 struct { - *baseTestSuite -} - -func (s *testSuite11) TestUpdateClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists t`) tk.MustExec(`create table t(id varchar(200) primary key, v int)`) @@ -386,10 +346,13 @@ func (s *testSuite11) TestUpdateClusterIndex(c *C) { tk.MustQuery("select * from s").Check(testkit.Rows("3 3 10", "5 5 5")) } -func (s *testSuite11) TestDeleteClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDeleteClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists t`) tk.MustExec(`create table t(id varchar(200) primary key, v int)`) @@ -421,10 +384,13 @@ func (s *testSuite11) TestDeleteClusterIndex(c *C) { tk.MustQuery("select * from s1").Check(testkit.Rows("5 5 5")) } -func (s *testSuite11) TestReplaceClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReplaceClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec(`drop table if exists rt1pk`) tk.MustExec(`create table rt1pk(id varchar(200) primary key, v int)`) @@ -448,38 +414,42 @@ func (s *testSuite11) TestReplaceClusterIndex(c *C) { tk.MustQuery(`select * from rt1pk1u`).Check(testkit.Rows("aaa 2 11")) } -func (s *testSuite11) TestPessimisticUpdatePKLazyCheck(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - s.testUpdatePKLazyCheck(c, tk, variable.ClusteredIndexDefModeOn) - s.testUpdatePKLazyCheck(c, tk, variable.ClusteredIndexDefModeOff) - s.testUpdatePKLazyCheck(c, tk, variable.ClusteredIndexDefModeIntOnly) +func TestPessimisticUpdatePKLazyCheck(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeOn) + testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeOff) + testUpdatePKLazyCheck(t, tk, variable.ClusteredIndexDefModeIntOnly) } -func (s *testSuite11) testUpdatePKLazyCheck(c *C, tk *testkit.TestKit, clusteredIndex variable.ClusteredIndexDefMode) { - tk.Se.GetSessionVars().EnableClusteredIndex = clusteredIndex +func testUpdatePKLazyCheck(t *testing.T, tk *testkit.TestKit, clusteredIndex variable.ClusteredIndexDefMode) { + tk.Session().GetSessionVars().EnableClusteredIndex = clusteredIndex tk.MustExec(`drop table if exists upk`) tk.MustExec(`create table upk (a int, b int, c int, primary key (a, b))`) tk.MustExec(`insert upk values (1, 1, 1), (2, 2, 2), (3, 3, 3)`) tk.MustExec("begin pessimistic") tk.MustExec("update upk set b = b + 1 where a between 1 and 2") - c.Assert(getPresumeExistsCount(c, tk.Se), Equals, 2) + require.Equal(t, 2, getPresumeExistsCount(t, tk.Session())) _, err := tk.Exec("update upk set a = 3, b = 3 where a between 1 and 2") - c.Assert(kv.ErrKeyExists.Equal(err), IsTrue) + require.True(t, kv.ErrKeyExists.Equal(err)) tk.MustExec("commit") } -func getPresumeExistsCount(c *C, se session.Session) int { +func getPresumeExistsCount(t *testing.T, se session.Session) int { txn, err := se.Txn(false) - c.Assert(err, IsNil) + require.NoError(t, err) buf := txn.GetMemBuffer() it, err := buf.Iter(nil, nil) - c.Assert(err, IsNil) + require.NoError(t, err) presumeNotExistsCnt := 0 for it.Valid() { flags, err1 := buf.GetFlags(it.Key()) - c.Assert(err1, IsNil) + require.Nil(t, err1) err = it.Next() - c.Assert(err, IsNil) + require.NoError(t, err) if flags.HasPresumeKeyNotExists() { presumeNotExistsCnt++ } @@ -487,18 +457,23 @@ func getPresumeExistsCount(c *C, se session.Session) int { return presumeNotExistsCnt } -func (s *testSuite11) TestOutOfRangeWithUnsigned(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOutOfRangeWithUnsigned(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t`) tk.MustExec(`create table t(ts int(10) unsigned NULL DEFAULT NULL)`) tk.MustExec(`insert into t values(1)`) _, err := tk.Exec("update t set ts = IF(ts < (0 - ts), 1,1) where ts>0") - c.Assert(err.Error(), Equals, "[types:1690]BIGINT UNSIGNED value is out of range in '(0 - test.t.ts)'") + require.Equal(t, "[types:1690]BIGINT UNSIGNED value is out of range in '(0 - test.t.ts)'", err.Error()) } -func (s *testPointGetSuite) TestIssue21447(c *C) { - tk1, tk2 := testkit.NewTestKit(c, s.store), testkit.NewTestKit(c, s.store) +func TestIssue21447(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk1, tk2 := testkit.NewTestKit(t, store), testkit.NewTestKit(t, store) tk1.MustExec("use test") tk2.MustExec("use test") @@ -521,8 +496,10 @@ func (s *testPointGetSuite) TestIssue21447(c *C) { tk1.MustExec("commit") } -func (s *testSuite11) TestIssue23553(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23553(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists tt`) tk.MustExec(`create table tt (m0 varchar(64), status tinyint not null)`) diff --git a/executor/window_test.go b/executor/window_test.go index 97ce36b4f3258..8b1c91132410d 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -16,13 +16,15 @@ package executor_test import ( "fmt" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" ) -func (s *testSuite7) TestWindowFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestWindowFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_window_concurrency = 1") tk.MustExec("set @@tidb_enable_pipelined_window_function = 0") defer func() { @@ -31,8 +33,10 @@ func (s *testSuite7) TestWindowFunctions(c *C) { doTestWindowFunctions(tk) } -func (s *testSuite7) TestWindowParallelFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestWindowParallelFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_window_concurrency = 4") tk.MustExec("set @@tidb_enable_pipelined_window_function = 0") defer func() { @@ -41,20 +45,23 @@ func (s *testSuite7) TestWindowParallelFunctions(c *C) { doTestWindowFunctions(tk) } -func (s *testSuite7) TestPipelinedWindowFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPipelinedWindowFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_window_concurrency = 1") doTestWindowFunctions(tk) } -func (s *testSuite7) TestPipelinedWindowParallelFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPipelinedWindowParallelFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_window_concurrency = 4") doTestWindowFunctions(tk) } func doTestWindowFunctions(tk *testkit.TestKit) { - var result *testkit.Result tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int)") @@ -63,129 +70,129 @@ func doTestWindowFunctions(tk *testkit.TestKit) { tk.MustExec("set @@tidb_enable_window_function = 0") }() tk.MustExec("insert into t values (1,2,3),(4,3,2),(2,3,4)") - result = tk.MustQuery("select count(a) over () from t") - result.Check(testkit.Rows("3", "3", "3")) - result = tk.MustQuery("select sum(a) over () + count(a) over () from t") - result.Check(testkit.Rows("10", "10", "10")) - result = tk.MustQuery("select sum(a) over (partition by a) from t").Sort() - result.Check(testkit.Rows("1", "2", "4")) - result = tk.MustQuery("select 1 + sum(a) over (), count(a) over () from t") - result.Check(testkit.Rows("8 3", "8 3", "8 3")) - result = tk.MustQuery("select sum(t1.a) over() from t t1, t t2") - result.Check(testkit.Rows("21", "21", "21", "21", "21", "21", "21", "21", "21")) - result = tk.MustQuery("select _tidb_rowid, sum(t.a) over() from t") - result.Check(testkit.Rows("1 7", "2 7", "3 7")) - - result = tk.MustQuery("select a, row_number() over() from t") - result.Check(testkit.Rows("1 1", "4 2", "2 3")) - result = tk.MustQuery("select a, row_number() over(partition by a) from t").Sort() - result.Check(testkit.Rows("1 1", "2 1", "4 1")) - - result = tk.MustQuery("select a, sum(a) over(rows between unbounded preceding and 1 following) from t") - result.Check(testkit.Rows("1 5", "4 7", "2 7")) - result = tk.MustQuery("select a, sum(a) over(rows between 1 preceding and 1 following) from t") - result.Check(testkit.Rows("1 5", "4 7", "2 6")) - result = tk.MustQuery("select a, sum(a) over(rows between unbounded preceding and 1 preceding) from t") - result.Check(testkit.Rows("1 ", "4 1", "2 5")) + tk.MustQuery("select count(a) over () from t"). + Check(testkit.Rows("3", "3", "3")) + tk.MustQuery("select sum(a) over () + count(a) over () from t"). + Check(testkit.Rows("10", "10", "10")) + tk.MustQuery("select sum(a) over (partition by a) from t").Sort(). + Check(testkit.Rows("1", "2", "4")) + tk.MustQuery("select 1 + sum(a) over (), count(a) over () from t"). + Check(testkit.Rows("8 3", "8 3", "8 3")) + tk.MustQuery("select sum(t1.a) over() from t t1, t t2"). + Check(testkit.Rows("21", "21", "21", "21", "21", "21", "21", "21", "21")) + tk.MustQuery("select _tidb_rowid, sum(t.a) over() from t"). + Check(testkit.Rows("1 7", "2 7", "3 7")) + + tk.MustQuery("select a, row_number() over() from t"). + Check(testkit.Rows("1 1", "4 2", "2 3")) + tk.MustQuery("select a, row_number() over(partition by a) from t").Sort(). + Check(testkit.Rows("1 1", "2 1", "4 1")) + + tk.MustQuery("select a, sum(a) over(rows between unbounded preceding and 1 following) from t"). + Check(testkit.Rows("1 5", "4 7", "2 7")) + tk.MustQuery("select a, sum(a) over(rows between 1 preceding and 1 following) from t"). + Check(testkit.Rows("1 5", "4 7", "2 6")) + tk.MustQuery("select a, sum(a) over(rows between unbounded preceding and 1 preceding) from t"). + Check(testkit.Rows("1 ", "4 1", "2 5")) tk.MustExec("drop table t") tk.MustExec("create table t(a int, b date)") tk.MustExec("insert into t values (null,null),(1,20190201),(2,20190202),(3,20190203),(5,20190205)") - result = tk.MustQuery("select a, sum(a) over(order by a range between 1 preceding and 2 following) from t") - result.Check(testkit.Rows(" ", "1 6", "2 6", "3 10", "5 5")) - result = tk.MustQuery("select a, sum(a) over(order by a desc range between 1 preceding and 2 following) from t") - result.Check(testkit.Rows("5 8", "3 6", "2 6", "1 3", " ")) - result = tk.MustQuery("select a, b, sum(a) over(order by b range between interval 1 day preceding and interval 2 day following) from t") - result.Check(testkit.Rows(" ", "1 2019-02-01 6", "2 2019-02-02 6", "3 2019-02-03 10", "5 2019-02-05 5")) - result = tk.MustQuery("select a, b, sum(a) over(order by b desc range between interval 1 day preceding and interval 2 day following) from t") - result.Check(testkit.Rows("5 2019-02-05 8", "3 2019-02-03 6", "2 2019-02-02 6", "1 2019-02-01 3", " ")) + tk.MustQuery("select a, sum(a) over(order by a range between 1 preceding and 2 following) from t"). + Check(testkit.Rows(" ", "1 6", "2 6", "3 10", "5 5")) + tk.MustQuery("select a, sum(a) over(order by a desc range between 1 preceding and 2 following) from t"). + Check(testkit.Rows("5 8", "3 6", "2 6", "1 3", " ")) + tk.MustQuery("select a, b, sum(a) over(order by b range between interval 1 day preceding and interval 2 day following) from t"). + Check(testkit.Rows(" ", "1 2019-02-01 6", "2 2019-02-02 6", "3 2019-02-03 10", "5 2019-02-05 5")) + tk.MustQuery("select a, b, sum(a) over(order by b desc range between interval 1 day preceding and interval 2 day following) from t"). + Check(testkit.Rows("5 2019-02-05 8", "3 2019-02-03 6", "2 2019-02-02 6", "1 2019-02-01 3", " ")) tk.MustExec("drop table t") tk.MustExec("CREATE TABLE t (id INTEGER, sex CHAR(1))") tk.MustExec("insert into t values (1, 'M'), (2, 'F'), (3, 'F'), (4, 'F'), (5, 'M'), (10, NULL), (11, NULL)") - result = tk.MustQuery("SELECT sex, id, RANK() OVER (PARTITION BY sex ORDER BY id DESC) FROM t").Sort() - result.Check(testkit.Rows(" 10 2", " 11 1", "F 2 3", "F 3 2", "F 4 1", "M 1 2", "M 5 1")) + tk.MustQuery("SELECT sex, id, RANK() OVER (PARTITION BY sex ORDER BY id DESC) FROM t").Sort(). + Check(testkit.Rows(" 10 2", " 11 1", "F 2 3", "F 3 2", "F 4 1", "M 1 2", "M 5 1")) tk.MustExec("drop table t") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values (1,1),(1,2),(2,1),(2,2)") - result = tk.MustQuery("select a, b, rank() over() from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) - result = tk.MustQuery("select a, b, rank() over(order by a) from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 1 3", "2 2 3")) - result = tk.MustQuery("select a, b, rank() over(order by a, b) from t") - result.Check(testkit.Rows("1 1 1", "1 2 2", "2 1 3", "2 2 4")) - - result = tk.MustQuery("select a, b, dense_rank() over() from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) - result = tk.MustQuery("select a, b, dense_rank() over(order by a) from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 1 2", "2 2 2")) - result = tk.MustQuery("select a, b, dense_rank() over(order by a, b) from t") - result.Check(testkit.Rows("1 1 1", "1 2 2", "2 1 3", "2 2 4")) - - result = tk.MustQuery("select row_number() over(rows between 1 preceding and 1 following) from t") - result.Check(testkit.Rows("1", "2", "3", "4")) - result = tk.MustQuery("show warnings") - result.Check(testkit.Rows("Note 3599 Window function 'row_number' ignores the frame clause of window '' and aggregates over the whole partition")) - - result = tk.MustQuery("select a, sum(a) over() from t") - result.Check(testkit.Rows("1 6", "1 6", "2 6", "2 6")) - result = tk.MustQuery("select a, sum(a) over(order by a) from t") - result.Check(testkit.Rows("1 2", "1 2", "2 6", "2 6")) - result = tk.MustQuery("select a, sum(a) over(order by a, b) from t") - result.Check(testkit.Rows("1 1", "1 2", "2 4", "2 6")) - - result = tk.MustQuery("select a, first_value(a) over(), last_value(a) over() from t") - result.Check(testkit.Rows("1 1 2", "1 1 2", "2 1 2", "2 1 2")) - result = tk.MustQuery("select a, first_value(a) over(rows between 1 preceding and 1 following), last_value(a) over(rows between 1 preceding and 1 following) from t") - result.Check(testkit.Rows("1 1 1", "1 1 2", "2 1 2", "2 2 2")) - result = tk.MustQuery("select a, first_value(a) over(rows between 1 following and 1 following), last_value(a) over(rows between 1 following and 1 following) from t") - result.Check(testkit.Rows("1 1 1", "1 2 2", "2 2 2", "2 ")) - result = tk.MustQuery("select a, first_value(rand(0)) over(), last_value(rand(0)) over() from t") - result.Check(testkit.Rows("1 0.15522042769493574 0.33109208227236947", "1 0.15522042769493574 0.33109208227236947", - "2 0.15522042769493574 0.33109208227236947", "2 0.15522042769493574 0.33109208227236947")) - - result = tk.MustQuery("select a, b, cume_dist() over() from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) - result = tk.MustQuery("select a, b, cume_dist() over(order by a) from t") - result.Check(testkit.Rows("1 1 0.5", "1 2 0.5", "2 1 1", "2 2 1")) - result = tk.MustQuery("select a, b, cume_dist() over(order by a, b) from t") - result.Check(testkit.Rows("1 1 0.25", "1 2 0.5", "2 1 0.75", "2 2 1")) - - result = tk.MustQuery("select a, nth_value(a, null) over() from t") - result.Check(testkit.Rows("1 ", "1 ", "2 ", "2 ")) - result = tk.MustQuery("select a, nth_value(a, 1) over() from t") - result.Check(testkit.Rows("1 1", "1 1", "2 1", "2 1")) - result = tk.MustQuery("select a, nth_value(a, 4) over() from t") - result.Check(testkit.Rows("1 2", "1 2", "2 2", "2 2")) - result = tk.MustQuery("select a, nth_value(a, 5) over() from t") - result.Check(testkit.Rows("1 ", "1 ", "2 ", "2 ")) - - result = tk.MustQuery("select ntile(3) over() from t") - result.Check(testkit.Rows("1", "1", "2", "3")) - result = tk.MustQuery("select ntile(2) over() from t") - result.Check(testkit.Rows("1", "1", "2", "2")) - result = tk.MustQuery("select ntile(null) over() from t") - result.Check(testkit.Rows("", "", "", "")) - - result = tk.MustQuery("select a, percent_rank() over() from t") - result.Check(testkit.Rows("1 0", "1 0", "2 0", "2 0")) - result = tk.MustQuery("select a, percent_rank() over(order by a) from t") - result.Check(testkit.Rows("1 0", "1 0", "2 0.6666666666666666", "2 0.6666666666666666")) - result = tk.MustQuery("select a, b, percent_rank() over(order by a, b) from t") - result.Check(testkit.Rows("1 1 0", "1 2 0.3333333333333333", "2 1 0.6666666666666666", "2 2 1")) - - result = tk.MustQuery("select a, lead(a) over (), lag(a) over() from t") - result.Check(testkit.Rows("1 1 ", "1 2 1", "2 2 1", "2 2")) - result = tk.MustQuery("select a, lead(a, 0) over(), lag(a, 0) over() from t") - result.Check(testkit.Rows("1 1 1", "1 1 1", "2 2 2", "2 2 2")) - result = tk.MustQuery("select a, lead(a, 1, a) over(), lag(a, 1, a) over() from t") - result.Check(testkit.Rows("1 1 1", "1 2 1", "2 2 1", "2 2 2")) - result = tk.MustQuery("select a, lead(a, 1, 'lead') over(), lag(a, 1, 'lag') over() from t") - result.Check(testkit.Rows("1 1 lag", "1 2 1", "2 2 1", "2 lead 2")) - - result = tk.MustQuery("SELECT CUME_DIST() OVER (ORDER BY null);") - result.Check(testkit.Rows("1")) + tk.MustQuery("select a, b, rank() over() from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) + tk.MustQuery("select a, b, rank() over(order by a) from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 1 3", "2 2 3")) + tk.MustQuery("select a, b, rank() over(order by a, b) from t"). + Check(testkit.Rows("1 1 1", "1 2 2", "2 1 3", "2 2 4")) + + tk.MustQuery("select a, b, dense_rank() over() from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) + tk.MustQuery("select a, b, dense_rank() over(order by a) from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 1 2", "2 2 2")) + tk.MustQuery("select a, b, dense_rank() over(order by a, b) from t"). + Check(testkit.Rows("1 1 1", "1 2 2", "2 1 3", "2 2 4")) + + tk.MustQuery("select row_number() over(rows between 1 preceding and 1 following) from t"). + Check(testkit.Rows("1", "2", "3", "4")) + tk.MustQuery("show warnings"). + Check(testkit.Rows("Note 3599 Window function 'row_number' ignores the frame clause of window '' and aggregates over the whole partition")) + + tk.MustQuery("select a, sum(a) over() from t"). + Check(testkit.Rows("1 6", "1 6", "2 6", "2 6")) + tk.MustQuery("select a, sum(a) over(order by a) from t"). + Check(testkit.Rows("1 2", "1 2", "2 6", "2 6")) + tk.MustQuery("select a, sum(a) over(order by a, b) from t"). + Check(testkit.Rows("1 1", "1 2", "2 4", "2 6")) + + tk.MustQuery("select a, first_value(a) over(), last_value(a) over() from t"). + Check(testkit.Rows("1 1 2", "1 1 2", "2 1 2", "2 1 2")) + tk.MustQuery("select a, first_value(a) over(rows between 1 preceding and 1 following), last_value(a) over(rows between 1 preceding and 1 following) from t"). + Check(testkit.Rows("1 1 1", "1 1 2", "2 1 2", "2 2 2")) + tk.MustQuery("select a, first_value(a) over(rows between 1 following and 1 following), last_value(a) over(rows between 1 following and 1 following) from t"). + Check(testkit.Rows("1 1 1", "1 2 2", "2 2 2", "2 ")) + tk.MustQuery("select a, first_value(rand(0)) over(), last_value(rand(0)) over() from t"). + Check(testkit.Rows("1 0.15522042769493574 0.33109208227236947", "1 0.15522042769493574 0.33109208227236947", + "2 0.15522042769493574 0.33109208227236947", "2 0.15522042769493574 0.33109208227236947")) + + tk.MustQuery("select a, b, cume_dist() over() from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 1 1", "2 2 1")) + tk.MustQuery("select a, b, cume_dist() over(order by a) from t"). + Check(testkit.Rows("1 1 0.5", "1 2 0.5", "2 1 1", "2 2 1")) + tk.MustQuery("select a, b, cume_dist() over(order by a, b) from t"). + Check(testkit.Rows("1 1 0.25", "1 2 0.5", "2 1 0.75", "2 2 1")) + + tk.MustQuery("select a, nth_value(a, null) over() from t"). + Check(testkit.Rows("1 ", "1 ", "2 ", "2 ")) + tk.MustQuery("select a, nth_value(a, 1) over() from t"). + Check(testkit.Rows("1 1", "1 1", "2 1", "2 1")) + tk.MustQuery("select a, nth_value(a, 4) over() from t"). + Check(testkit.Rows("1 2", "1 2", "2 2", "2 2")) + tk.MustQuery("select a, nth_value(a, 5) over() from t"). + Check(testkit.Rows("1 ", "1 ", "2 ", "2 ")) + + tk.MustQuery("select ntile(3) over() from t"). + Check(testkit.Rows("1", "1", "2", "3")) + tk.MustQuery("select ntile(2) over() from t"). + Check(testkit.Rows("1", "1", "2", "2")) + tk.MustQuery("select ntile(null) over() from t"). + Check(testkit.Rows("", "", "", "")) + + tk.MustQuery("select a, percent_rank() over() from t"). + Check(testkit.Rows("1 0", "1 0", "2 0", "2 0")) + tk.MustQuery("select a, percent_rank() over(order by a) from t"). + Check(testkit.Rows("1 0", "1 0", "2 0.6666666666666666", "2 0.6666666666666666")) + tk.MustQuery("select a, b, percent_rank() over(order by a, b) from t"). + Check(testkit.Rows("1 1 0", "1 2 0.3333333333333333", "2 1 0.6666666666666666", "2 2 1")) + + tk.MustQuery("select a, lead(a) over (), lag(a) over() from t"). + Check(testkit.Rows("1 1 ", "1 2 1", "2 2 1", "2 2")) + tk.MustQuery("select a, lead(a, 0) over(), lag(a, 0) over() from t"). + Check(testkit.Rows("1 1 1", "1 1 1", "2 2 2", "2 2 2")) + tk.MustQuery("select a, lead(a, 1, a) over(), lag(a, 1, a) over() from t"). + Check(testkit.Rows("1 1 1", "1 2 1", "2 2 1", "2 2 2")) + tk.MustQuery("select a, lead(a, 1, 'lead') over(), lag(a, 1, 'lag') over() from t"). + Check(testkit.Rows("1 1 lag", "1 2 1", "2 2 1", "2 lead 2")) + + tk.MustQuery("SELECT CUME_DIST() OVER (ORDER BY null);"). + Check(testkit.Rows("1")) tk.MustQuery("select lead(a) over(partition by null) from t").Sort().Check(testkit.Rows("1", "2", "2", "")) @@ -207,43 +214,47 @@ func doTestWindowFunctions(tk *testkit.TestKit) { testkit.Rows(" ", "1.00 ", "2.00 "), ) - result = tk.MustQuery("select sum(a) over w, sum(b) over w from t window w as (order by a)") - result.Check(testkit.Rows("2 3", "2 3", "6 6", "6 6")) - result = tk.MustQuery("select row_number() over w, sum(b) over w from t window w as (order by a)") - result.Check(testkit.Rows("1 3", "2 3", "3 6", "4 6")) - result = tk.MustQuery("select row_number() over w, sum(b) over w from t window w as (rows between 1 preceding and 1 following)") - result.Check(testkit.Rows("1 3", "2 4", "3 5", "4 3")) + tk.MustQuery("select sum(a) over w, sum(b) over w from t window w as (order by a)"). + Check(testkit.Rows("2 3", "2 3", "6 6", "6 6")) + tk.MustQuery("select row_number() over w, sum(b) over w from t window w as (order by a)"). + Check(testkit.Rows("1 3", "2 3", "3 6", "4 6")) + tk.MustQuery("select row_number() over w, sum(b) over w from t window w as (rows between 1 preceding and 1 following)"). + Check(testkit.Rows("1 3", "2 4", "3 5", "4 3")) - tk.Se.GetSessionVars().MaxChunkSize = 1 - result = tk.MustQuery("select a, row_number() over (partition by a) from t").Sort() - result.Check(testkit.Rows("1 1", "1 2", "2 1", "2 2")) + tk.Session().GetSessionVars().MaxChunkSize = 1 + tk.MustQuery("select a, row_number() over (partition by a) from t").Sort(). + Check(testkit.Rows("1 1", "1 2", "2 1", "2 2")) } -func (s *testSuite7) TestWindowFunctionsDataReference(c *C) { +func TestWindowFunctionsDataReference(t *testing.T) { // see https://github.com/pingcap/tidb/issues/11614 - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values (2,1),(2,2),(2,3)") - tk.Se.GetSessionVars().MaxChunkSize = 2 - result := tk.MustQuery("select a, b, rank() over (partition by a order by b) from t") - result.Check(testkit.Rows("2 1 1", "2 2 2", "2 3 3")) - result = tk.MustQuery("select a, b, PERCENT_RANK() over (partition by a order by b) from t") - result.Check(testkit.Rows("2 1 0", "2 2 0.5", "2 3 1")) - result = tk.MustQuery("select a, b, CUME_DIST() over (partition by a order by b) from t") - result.Check(testkit.Rows("2 1 0.3333333333333333", "2 2 0.6666666666666666", "2 3 1")) + tk.Session().GetSessionVars().MaxChunkSize = 2 + tk.MustQuery("select a, b, rank() over (partition by a order by b) from t"). + Check(testkit.Rows("2 1 1", "2 2 2", "2 3 3")) + tk.MustQuery("select a, b, PERCENT_RANK() over (partition by a order by b) from t"). + Check(testkit.Rows("2 1 0", "2 2 0.5", "2 3 1")) + tk.MustQuery("select a, b, CUME_DIST() over (partition by a order by b) from t"). + Check(testkit.Rows("2 1 0.3333333333333333", "2 2 0.6666666666666666", "2 3 1")) // see https://github.com/pingcap/tidb/issues/12415 - result = tk.MustQuery("select b, first_value(b) over (order by b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) from t") - result.Check(testkit.Rows("1 1", "2 1", "3 1")) - result = tk.MustQuery("select b, first_value(b) over (order by b ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) from t") - result.Check(testkit.Rows("1 1", "2 1", "3 1")) + tk.MustQuery("select b, first_value(b) over (order by b RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) from t"). + Check(testkit.Rows("1 1", "2 1", "3 1")) + tk.MustQuery("select b, first_value(b) over (order by b ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) from t"). + Check(testkit.Rows("1 1", "2 1", "3 1")) } -func (s *testSuite7) TestSlidingWindowFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSlidingWindowFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set @@tidb_enable_pipelined_window_function=0;") defer func() { @@ -261,8 +272,10 @@ func (s *testSuite7) TestSlidingWindowFunctions(c *C) { } } -func (s *testSuite7) TestPipelinedSlidingWindowFunctions(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPipelinedSlidingWindowFunctions(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") idTypes := []string{"FLOAT", "DOUBLE"} useHighPrecisions := []string{"ON", "OFF"} @@ -277,7 +290,6 @@ func (s *testSuite7) TestPipelinedSlidingWindowFunctions(c *C) { } func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { - var result *testkit.Result tk.MustExec("insert into t values (1,'M')") tk.MustExec("insert into t values (2,'F')") tk.MustExec("insert into t values (3,'F')") @@ -289,163 +301,165 @@ func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { tk.MustExec("PREPARE p FROM 'SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN ? PRECEDING and ? PRECEDING) FROM t';") tk.MustExec("SET @p1= 1;") tk.MustExec("SET @p2= 2;") - result = tk.MustQuery("EXECUTE p USING @p1, @p2;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("EXECUTE p USING @p2, @p1;") - result.Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 2", " 2")) + tk.MustQuery("EXECUTE p USING @p1, @p2;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("EXECUTE p USING @p2, @p1;"). + Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 2", " 2")) tk.MustExec("DROP PREPARE p;") tk.MustExec("PREPARE p FROM 'SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN ? FOLLOWING and ? FOLLOWING) FROM t';") tk.MustExec("SET @p1= 1;") tk.MustExec("SET @p2= 2;") - result = tk.MustQuery("EXECUTE p USING @p2, @p1;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("EXECUTE p USING @p1, @p2;") - result.Check(testkit.Rows("M 2", "F 2", "F 2", "F 2", "M 2", " 1", " 0")) + tk.MustQuery("EXECUTE p USING @p2, @p1;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("EXECUTE p USING @p1, @p2;"). + Check(testkit.Rows("M 2", "F 2", "F 2", "F 2", "M 2", " 1", " 0")) tk.MustExec("DROP PREPARE p;") // COUNT ROWS - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 2", "F 2", "F 2", "M 2", " 1", " 0")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 2", " 2")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 2", "F 2", "F 2", "M 2", " 1", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 2", " 2")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) // COUNT RANGE - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 2", "F 2", "F 1", "M 0", " 1", " 0")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 0", " 1")) - result = tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 2", "F 2", "F 1", "M 0", " 1", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 1", "F 2", "F 2", "M 2", " 0", " 1")) + tk.MustQuery("SELECT sex, COUNT(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) // SUM ROWS - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 5", "F 7", "F 9", "F 15", "M 21", " 11", " ")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 3", "F 5", "M 7", " 9", " 15")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 3", "F 6", "F 10", "F 15", "M 25", " 36", " 36")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 5", "F 7", "F 9", "F 15", "M 21", " 11", " ")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 3", "F 5", "M 7", " 9", " 15")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 3", "F 6", "F 10", "F 15", "M 25", " 36", " 36")) // SUM RANGE - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 5", "F 7", "F 9", "F 5", "M ", " 11", " ")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 3", "F 5", "M 7", " ", " 10")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 6", "F 10", "F 14", "F 12", "M 9", " 21", " 21")) - result = tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows(" 21", " 21", "M 12", "F 14", "F 10", "F 6", "M 3")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 5", "F 7", "F 9", "F 5", "M ", " 11", " ")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 3", "F 5", "M 7", " ", " 10")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 6", "F 10", "F 14", "F 12", "M 9", " 21", " 21")) + tk.MustQuery("SELECT sex, SUM(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows(" 21", " 21", "M 12", "F 14", "F 10", "F 6", "M 3")) // AVG ROWS - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2.5", "F 3.5", "F 4.5", "F 7.5", "M 10.5", " 11", " ")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 1.5", "F 2.5", "M 3.5", " 4.5", " 7.5")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 1.5", "F 2", "F 2.5", "F 3", "M 4.166666666666667", " 5.142857142857143", " 5.142857142857143")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2.5", "F 3.5", "F 4.5", "F 7.5", "M 10.5", " 11", " ")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 1.5", "F 2.5", "M 3.5", " 4.5", " 7.5")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 1.5", "F 2", "F 2.5", "F 3", "M 4.166666666666667", " 5.142857142857143", " 5.142857142857143")) // AVG RANGE - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2.5", "F 3.5", "F 4.5", "F 5", "M ", " 11", " ")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 1.5", "F 2.5", "M 3.5", " ", " 10")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 2.5", "F 3.5", "F 4", "M 4.5", " 10.5", " 10.5")) - result = tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows(" 10.5", " 10.5", "M 4", "F 3.5", "F 2.5", "F 2", "M 1.5")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2.5", "F 3.5", "F 4.5", "F 5", "M ", " 11", " ")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 1.5", "F 2.5", "M 3.5", " ", " 10")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 2.5", "F 3.5", "F 4", "M 4.5", " 10.5", " 10.5")) + tk.MustQuery("SELECT sex, AVG(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows(" 10.5", " 10.5", "M 4", "F 3.5", "F 2.5", "F 2", "M 1.5")) // BIT_XOR ROWS - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 1", "F 7", "F 1", "F 15", "M 1", " 11", " 0")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 1", " 15")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 3", "F 0", "F 4", "F 1", "M 11", " 0", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 1", "F 7", "F 1", "F 15", "M 1", " 11", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 1", " 15")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 3", "F 0", "F 4", "F 1", "M 11", " 0", " 0")) // BIT_XOR RANGE - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 1", "F 7", "F 1", "F 5", "M 0", " 11", " 0")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 0", " 10")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 0", "F 4", "F 0", "F 2", "M 1", " 1", " 1")) - result = tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows(" 1", " 1", "M 2", "F 0", "F 4", "F 0", "M 3")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 1", "F 7", "F 1", "F 5", "M 0", " 11", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 0", "F 0", "F 0", "F 0", "M 0", " 0", " 0")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M 0", "F 1", "F 3", "F 1", "M 7", " 0", " 10")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 0", "F 4", "F 0", "F 2", "M 1", " 1", " 1")) + tk.MustQuery("SELECT sex, BIT_XOR(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows(" 1", " 1", "M 2", "F 0", "F 4", "F 0", "M 3")) // MIN ROWS - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M 10", " 11", " ")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 1", "F 2", "M 3", " 4", " 5")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 1", "F 1", "F 1", "F 1", "M 1", " 1", " 1")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M 10", " 11", " ")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 1", "F 2", "M 3", " 4", " 5")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 1", "F 1", "F 1", "F 1", "M 1", " 1", " 1")) // MIN RANGE - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M ", " 11", " ")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 1", "F 2", "M 3", " ", " 10")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 1", "F 1", "F 2", "F 3", "M 4", " 10", " 10")) - result = tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows(" 10", " 10", "M 3", "F 2", "F 1", "F 1", "M 1")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M ", " 11", " ")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 1", "F 2", "M 3", " ", " 10")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 1", "F 1", "F 2", "F 3", "M 4", " 10", " 10")) + tk.MustQuery("SELECT sex, MIN(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows(" 10", " 10", "M 3", "F 2", "F 1", "F 1", "M 1")) // MAX ROWS - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 3", "F 4", "F 5", "F 10", "M 11", " 11", " ")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 2", "F 3", "M 4", " 5", " 10")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M 10", " 11", " 11")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 3", "F 4", "F 5", "F 10", "M 11", " 11", " ")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 2", "F 3", "M 4", " 5", " 10")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN 1 PRECEDING and 3 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 2", "F 3", "F 4", "F 5", "M 10", " 11", " 11")) // MAX RANGE - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 3", "F 4", "F 5", "F 5", "M ", " 11", " ")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;") - result.Check(testkit.Rows("M ", "F 1", "F 2", "F 3", "M 4", " ", " 10")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows("M 3", "F 4", "F 5", "F 5", "M 5", " 11", " 11")) - result = tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;") - result.Check(testkit.Rows(" 11", " 11", "M 5", "F 5", "F 4", "F 3", "M 2")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 1 FOLLOWING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 3", "F 4", "F 5", "F 5", "M ", " 11", " ")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 3 FOLLOWING and 1 FOLLOWING) FROM t;"). + Check(testkit.Rows("M ", "F ", "F ", "F ", "M ", " ", " ")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 2 PRECEDING and 1 PRECEDING) FROM t;"). + Check(testkit.Rows("M ", "F 1", "F 2", "F 3", "M 4", " ", " 10")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows("M 3", "F 4", "F 5", "F 5", "M 5", " 11", " 11")) + tk.MustQuery("SELECT sex, MAX(id) OVER (ORDER BY id DESC RANGE BETWEEN 1 PRECEDING and 2 FOLLOWING) FROM t;"). + Check(testkit.Rows(" 11", " 11", "M 5", "F 5", "F 4", "F 3", "M 2")) } -func (s *testSuite7) TestIssue24264(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24264(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tbl_2") tk.MustExec("create table tbl_2 ( col_10 char(65) collate utf8mb4_unicode_ci not null , col_11 bigint not null , col_12 datetime not null , col_13 bigint unsigned default 327695751717730004 , col_14 timestamp default '2010-11-18' not null , primary key idx_5 ( col_11,col_13 ) /*T![clustered_index] clustered */ , unique key idx_6 ( col_10,col_11,col_13 ) , unique key idx_7 ( col_14,col_12,col_13 ) )") @@ -472,8 +486,10 @@ func (s *testSuite7) TestIssue24264(c *C) { "")) } -func (s *testSuite7) TestIssue29947(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29947(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t_tir89b, t_vejdy`) diff --git a/executor/write.go b/executor/write.go index 9b690d1141382..62013156e63d6 100644 --- a/executor/write.go +++ b/executor/write.go @@ -159,7 +159,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old // 4. Fill values into on-update-now fields, only if they are really changed. for i, col := range t.Cols() { if mysql.HasOnUpdateNowFlag(col.Flag) && !modified[i] && !onUpdateSpecified[i] { - if v, err := expression.GetTimeValue(sctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, int8(col.Decimal)); err == nil { + if v, err := expression.GetTimeValue(sctx, strings.ToUpper(ast.CurrentTimestamp), col.Tp, col.Decimal); err == nil { newData[i] = v modified[i] = true } else { diff --git a/executor/write_test.go b/executor/write_test.go index 2abd1e7a10563..04b3864337744 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" @@ -1139,8 +1138,6 @@ func TestReplace(t *testing.T) { } func TestReplaceWithCICollation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -4256,9 +4253,6 @@ func TestListColumnsPartitionWithGlobalIndex(t *testing.T) { } func TestIssue20724(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -4272,9 +4266,6 @@ func TestIssue20724(t *testing.T) { } func TestIssue20840(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) @@ -4289,9 +4280,6 @@ func TestIssue20840(t *testing.T) { } func TestIssueInsertPrefixIndexForNonUTF8Collation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) diff --git a/expression/aggregation/agg_to_pb.go b/expression/aggregation/agg_to_pb.go index d50c7ab4de78c..a2e5b89539acc 100644 --- a/expression/aggregation/agg_to_pb.go +++ b/expression/aggregation/agg_to_pb.go @@ -102,9 +102,48 @@ func AggFuncToPBExpr(sctx sessionctx.Context, client kv.Client, aggFunc *AggFunc sc.AppendWarning(errors.Errorf("Error happened when buildGroupConcat: %s", err.Error())) return nil } - return &tipb.Expr{Tp: tp, Val: codec.EncodeUint(nil, maxLen), Children: children, FieldType: expression.ToPBFieldType(aggFunc.RetTp), HasDistinct: aggFunc.HasDistinct, OrderBy: orderBy} + return &tipb.Expr{Tp: tp, Val: codec.EncodeUint(nil, maxLen), Children: children, FieldType: expression.ToPBFieldType(aggFunc.RetTp), HasDistinct: aggFunc.HasDistinct, OrderBy: orderBy, AggFuncMode: AggFunctionModeToPB(aggFunc.Mode)} } - return &tipb.Expr{Tp: tp, Children: children, FieldType: expression.ToPBFieldType(aggFunc.RetTp), HasDistinct: aggFunc.HasDistinct} + return &tipb.Expr{Tp: tp, Children: children, FieldType: expression.ToPBFieldType(aggFunc.RetTp), HasDistinct: aggFunc.HasDistinct, AggFuncMode: AggFunctionModeToPB(aggFunc.Mode)} +} + +// AggFunctionModeToPB converts aggregate function mode to PB. +func AggFunctionModeToPB(mode AggFunctionMode) (pbMode *tipb.AggFunctionMode) { + pbMode = new(tipb.AggFunctionMode) + switch mode { + case CompleteMode: + *pbMode = tipb.AggFunctionMode_CompleteMode + case FinalMode: + *pbMode = tipb.AggFunctionMode_FinalMode + case Partial1Mode: + *pbMode = tipb.AggFunctionMode_Partial1Mode + case Partial2Mode: + *pbMode = tipb.AggFunctionMode_Partial2Mode + case DedupMode: + *pbMode = tipb.AggFunctionMode_DedupMode + } + return pbMode +} + +// PBAggFuncModeToAggFuncMode converts pb to aggregate function mode. +func PBAggFuncModeToAggFuncMode(pbMode *tipb.AggFunctionMode) (mode AggFunctionMode) { + // Default mode of the aggregate function is PartialMode. + mode = Partial1Mode + if pbMode != nil { + switch *pbMode { + case tipb.AggFunctionMode_CompleteMode: + mode = CompleteMode + case tipb.AggFunctionMode_FinalMode: + mode = FinalMode + case tipb.AggFunctionMode_Partial1Mode: + mode = Partial1Mode + case tipb.AggFunctionMode_Partial2Mode: + mode = Partial2Mode + case tipb.AggFunctionMode_DedupMode: + mode = DedupMode + } + } + return mode } // PBExprToAggFuncDesc converts pb to aggregate function. @@ -149,7 +188,7 @@ func PBExprToAggFuncDesc(ctx sessionctx.Context, aggFunc *tipb.Expr, fieldTps [] base.WrapCastForAggArgs(ctx) return &AggFuncDesc{ baseFuncDesc: base, - Mode: Partial1Mode, + Mode: PBAggFuncModeToAggFuncMode(aggFunc.AggFuncMode), HasDistinct: false, }, nil } diff --git a/expression/aggregation/agg_to_pb_test.go b/expression/aggregation/agg_to_pb_test.go index 9aa886e2d70ea..31558d16c7467 100644 --- a/expression/aggregation/agg_to_pb_test.go +++ b/expression/aggregation/agg_to_pb_test.go @@ -51,13 +51,13 @@ func TestAggFunc2Pb(t *testing.T) { } jsons := []string{ - `{"tp":3002,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, - `{"tp":3001,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":8,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, - `{"tp":3003,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, - `{"tp":3007,"val":"AAAAAAAABAA=","children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":15,"flag":0,"flen":-1,"decimal":-1,"collate":46,"charset":"utf8mb4"},"has_distinct":%v}`, - `{"tp":3005,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, - `{"tp":3004,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, - `{"tp":3006,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":63,"charset":"binary"},"has_distinct":%v}`, + `{"tp":3002,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3001,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":8,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3003,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3007,"val":"AAAAAAAABAA=","children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":15,"flag":0,"flen":-1,"decimal":-1,"collate":-46,"charset":"utf8mb4"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3005,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3004,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, + `{"tp":3006,"children":[{"tp":201,"val":"gAAAAAAAAAE=","sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":false}],"sig":0,"field_type":{"tp":5,"flag":0,"flen":-1,"decimal":-1,"collate":-63,"charset":"binary"},"has_distinct":%v,"aggFuncMode":0}`, } for i, funcName := range funcNames { for _, hasDistinct := range []bool{true, false} { diff --git a/expression/builtin.go b/expression/builtin.go index 71fde7647a598..b693d21145223 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -187,21 +187,21 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Ex fieldType = &types.FieldType{ Tp: mysql.TypeDatetime, Flen: mysql.MaxDatetimeWidthWithFsp, - Decimal: int(types.MaxFsp), + Decimal: types.MaxFsp, Flag: mysql.BinaryFlag, } case types.ETTimestamp: fieldType = &types.FieldType{ Tp: mysql.TypeTimestamp, Flen: mysql.MaxDatetimeWidthWithFsp, - Decimal: int(types.MaxFsp), + Decimal: types.MaxFsp, Flag: mysql.BinaryFlag, } case types.ETDuration: fieldType = &types.FieldType{ Tp: mysql.TypeDuration, Flen: mysql.MaxDurationWidthWithFsp, - Decimal: int(types.MaxFsp), + Decimal: types.MaxFsp, Flag: mysql.BinaryFlag, } case types.ETJson: @@ -781,6 +781,7 @@ var funcs = map[string]functionClass{ ast.VitessHash: &vitessHashFunctionClass{baseFunctionClass{ast.VitessHash, 1, 1}}, ast.UUIDToBin: &uuidToBinFunctionClass{baseFunctionClass{ast.UUIDToBin, 1, 2}}, ast.BinToUUID: &binToUUIDFunctionClass{baseFunctionClass{ast.BinToUUID, 1, 2}}, + ast.TiDBShard: &tidbShardFunctionClass{baseFunctionClass{ast.TiDBShard, 1, 1}}, // get_lock() and release_lock() are parsed but do nothing. // It is used for preventing error in Ruby's activerecord migrations. diff --git a/expression/builtin_arithmetic.go b/expression/builtin_arithmetic.go index 6c82a938206ce..e76bec6c157ec 100644 --- a/expression/builtin_arithmetic.go +++ b/expression/builtin_arithmetic.go @@ -128,10 +128,10 @@ func setFlenDecimal4RealOrDecimal(ctx sessionctx.Context, retTp *types.FieldType func (c *arithmeticDivideFunctionClass) setType4DivDecimal(retTp, a, b *types.FieldType) { var deca, decb = a.Decimal, b.Decimal - if deca == int(types.UnspecifiedFsp) { + if deca == types.UnspecifiedFsp { deca = 0 } - if decb == int(types.UnspecifiedFsp) { + if decb == types.UnspecifiedFsp { decb = 0 } retTp.Decimal = deca + precIncrement diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 6ee9d49ed9123..9e5eefa03d97b 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -589,7 +589,7 @@ func (b *builtinCastIntAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNul if b.args[0].GetType().Tp == mysql.TypeYear { res, err = types.ParseTimeFromYear(b.ctx.GetSessionVars().StmtCtx, val) } else { - res, err = types.ParseTimeFromNum(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Tp, int8(b.tp.Decimal)) + res, err = types.ParseTimeFromNum(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Tp, b.tp.Decimal) } if err != nil { @@ -617,7 +617,7 @@ func (b *builtinCastIntAsDurationSig) evalDuration(row chunk.Row) (res types.Dur if isNull || err != nil { return res, isNull, err } - dur, err := types.NumberToDuration(val, int8(b.tp.Decimal)) + dur, err := types.NumberToDuration(val, b.tp.Decimal) if err != nil { if types.ErrOverflow.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleOverflow(err, err) @@ -890,7 +890,7 @@ func (b *builtinCastRealAsTimeSig) evalTime(row chunk.Row) (types.Time, bool, er return types.ZeroTime, false, nil } sc := b.ctx.GetSessionVars().StmtCtx - res, err := types.ParseTime(sc, fv, b.tp.Tp, int8(b.tp.Decimal)) + res, err := types.ParseTime(sc, fv, b.tp.Tp, b.tp.Decimal) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -916,7 +916,7 @@ func (b *builtinCastRealAsDurationSig) evalDuration(row chunk.Row) (res types.Du if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), int8(b.tp.Decimal)) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), b.tp.Decimal) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1068,7 +1068,7 @@ func (b *builtinCastDecimalAsTimeSig) evalTime(row chunk.Row) (res types.Time, i return res, isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTimeFromFloatString(sc, string(val.ToString()), b.tp.Tp, int8(b.tp.Decimal)) + res, err = types.ParseTimeFromFloatString(sc, string(val.ToString()), b.tp.Tp, b.tp.Decimal) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -1094,7 +1094,7 @@ func (b *builtinCastDecimalAsDurationSig) evalDuration(row chunk.Row) (res types if isNull || err != nil { return res, true, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), int8(b.tp.Decimal)) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(val.ToString()), b.tp.Decimal) if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) // ErrTruncatedWrongVal needs to be considered NULL. @@ -1288,7 +1288,7 @@ func (b *builtinCastStringAsTimeSig) evalTime(row chunk.Row) (res types.Time, is return res, isNull, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTime(sc, val, b.tp.Tp, int8(b.tp.Decimal)) + res, err = types.ParseTime(sc, val, b.tp.Tp, b.tp.Decimal) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -1317,7 +1317,7 @@ func (b *builtinCastStringAsDurationSig) evalDuration(row chunk.Row) (res types. if isNull || err != nil { return res, isNull, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, int8(b.tp.Decimal)) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, val, b.tp.Decimal) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) @@ -1349,7 +1349,7 @@ func (b *builtinCastTimeAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu if res, err = res.Convert(sc, b.tp.Tp); err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } - res, err = res.RoundFrac(sc, int8(b.tp.Decimal)) + res, err = res.RoundFrac(sc, b.tp.Decimal) if b.tp.Tp == mysql.TypeDate { // Truncate hh:mm:ss part if the type is Date. res.SetCoreTime(types.FromDate(res.Year(), res.Month(), res.Day(), 0, 0, 0, 0)) @@ -1463,7 +1463,7 @@ func (b *builtinCastTimeAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = res.RoundFrac(int8(b.tp.Decimal), b.ctx.GetSessionVars().Location()) + res, err = res.RoundFrac(b.tp.Decimal, b.ctx.GetSessionVars().Location()) return res, false, err } @@ -1482,7 +1482,7 @@ func (b *builtinCastDurationAsDurationSig) evalDuration(row chunk.Row) (res type if isNull || err != nil { return res, isNull, err } - res, err = res.RoundFrac(int8(b.tp.Decimal), b.ctx.GetSessionVars().Location()) + res, err = res.RoundFrac(b.tp.Decimal, b.ctx.GetSessionVars().Location()) return res, false, err } @@ -1524,7 +1524,7 @@ func (b *builtinCastDurationAsRealSig) evalReal(row chunk.Row) (res float64, isN if isNull || err != nil { return res, isNull, err } - if val.Fsp, err = types.CheckFsp(int(val.Fsp)); err != nil { + if val.Fsp, err = types.CheckFsp(val.Fsp); err != nil { return res, false, err } res, err = val.ToNumber().ToFloat64() @@ -1546,7 +1546,7 @@ func (b *builtinCastDurationAsDecimalSig) evalDecimal(row chunk.Row) (res *types if isNull || err != nil { return res, isNull, err } - if val.Fsp, err = types.CheckFsp(int(val.Fsp)); err != nil { + if val.Fsp, err = types.CheckFsp(val.Fsp); err != nil { return res, false, err } sc := b.ctx.GetSessionVars().StmtCtx @@ -1614,7 +1614,7 @@ func (b *builtinCastDurationAsTimeSig) evalTime(row chunk.Row) (res types.Time, if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } - res, err = res.RoundFrac(sc, int8(b.tp.Decimal)) + res, err = res.RoundFrac(sc, b.tp.Decimal) return res, false, err } @@ -1734,7 +1734,7 @@ func (b *builtinCastJSONAsTimeSig) evalTime(row chunk.Row) (res types.Time, isNu return res, false, err } sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseTime(sc, s, b.tp.Tp, int8(b.tp.Decimal)) + res, err = types.ParseTime(sc, s, b.tp.Tp, b.tp.Decimal) if err != nil { return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) } @@ -1764,7 +1764,7 @@ func (b *builtinCastJSONAsDurationSig) evalDuration(row chunk.Row) (res types.Du if err != nil { return res, false, err } - res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, int8(b.tp.Decimal)) + res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, s, b.tp.Decimal) if types.ErrTruncatedWrongVal.Equal(err) { sc := b.ctx.GetSessionVars().StmtCtx err = sc.HandleTruncate(err) @@ -1952,7 +1952,7 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression { // into consideration, so we set `expr.GetType().Flen + 3` as the `argLen`. // Since the length of float and double is not accurate, we do not handle // them. - if exprTp.Tp == mysql.TypeNewDecimal && argLen != int(types.UnspecifiedFsp) { + if exprTp.Tp == mysql.TypeNewDecimal && argLen != types.UnspecifiedFsp { argLen += 3 } if exprTp.EvalType() == types.ETInt { @@ -1983,15 +1983,15 @@ func WrapWithCastAsTime(ctx sessionctx.Context, expr Expression, tp *types.Field } switch x := expr.GetType().EvalType(); x { case types.ETInt: - tp.Decimal = int(types.MinFsp) + tp.Decimal = types.MinFsp case types.ETString, types.ETReal, types.ETJson: - tp.Decimal = int(types.MaxFsp) + tp.Decimal = types.MaxFsp case types.ETDatetime, types.ETTimestamp, types.ETDuration: tp.Decimal = expr.GetType().Decimal case types.ETDecimal: tp.Decimal = expr.GetType().Decimal - if tp.Decimal > int(types.MaxFsp) { - tp.Decimal = int(types.MaxFsp) + if tp.Decimal > types.MaxFsp { + tp.Decimal = types.MaxFsp } default: } @@ -2019,7 +2019,7 @@ func WrapWithCastAsDuration(ctx sessionctx.Context, expr Expression) Expression case mysql.TypeDatetime, mysql.TypeTimestamp, mysql.TypeDate: tp.Decimal = x.Decimal default: - tp.Decimal = int(types.MaxFsp) + tp.Decimal = types.MaxFsp } tp.Flen = mysql.MaxDurationWidthNoFsp if tp.Decimal > 0 { diff --git a/expression/builtin_cast_test.go b/expression/builtin_cast_test.go index 43958afe78ec8..80547248e4114 100644 --- a/expression/builtin_cast_test.go +++ b/expression/builtin_cast_test.go @@ -323,7 +323,7 @@ func TestCastFuncSig(t *testing.T) { var sig builtinFunc durationColumn := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} - durationColumn.RetType.Decimal = int(types.DefaultFsp) + durationColumn.RetType.Decimal = types.DefaultFsp // Test cast as Decimal. castToDecCases := []struct { before *Column @@ -817,7 +817,7 @@ func TestCastFuncSig(t *testing.T) { for i, c := range castToTimeCases { args := []Expression{c.before} tp := types.NewFieldType(mysql.TypeDatetime) - tp.Decimal = int(types.DefaultFsp) + tp.Decimal = types.DefaultFsp timeFunc, err := newBaseBuiltinFunc(ctx, "", args, 0) require.NoError(t, err) timeFunc.tp = tp @@ -846,7 +846,7 @@ func TestCastFuncSig(t *testing.T) { castToTimeCases2 := []struct { before *Column after types.Time - fsp int8 + fsp int tp byte row chunk.MutRow }{ @@ -902,7 +902,7 @@ func TestCastFuncSig(t *testing.T) { for i, c := range castToTimeCases2 { args := []Expression{c.before} tp := types.NewFieldType(c.tp) - tp.Decimal = int(c.fsp) + tp.Decimal = c.fsp timeFunc, err := newBaseBuiltinFunc(ctx, "", args, 0) require.NoError(t, err) timeFunc.tp = tp @@ -926,7 +926,7 @@ func TestCastFuncSig(t *testing.T) { resAfter := c.after.String() if c.fsp > 0 { resAfter += "." - for i := 0; i < int(c.fsp); i++ { + for i := 0; i < c.fsp; i++ { resAfter += "0" } } @@ -984,7 +984,7 @@ func TestCastFuncSig(t *testing.T) { for i, c := range castToDurationCases { args := []Expression{c.before} tp := types.NewFieldType(mysql.TypeDuration) - tp.Decimal = int(types.DefaultFsp) + tp.Decimal = types.DefaultFsp durationFunc, err := newBaseBuiltinFunc(ctx, "", args, 0) require.NoError(t, err) durationFunc.tp = tp @@ -1166,7 +1166,7 @@ func TestWrapWithCastAsTypesClasses(t *testing.T) { ctx := createContext(t) durationColumn0 := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} - durationColumn0.RetType.Decimal = int(types.DefaultFsp) + durationColumn0.RetType.Decimal = types.DefaultFsp durationColumn3 := &Column{RetType: types.NewFieldType(mysql.TypeDuration), Index: 0} durationColumn3.RetType.Decimal = 3 cases := []struct { diff --git a/expression/builtin_cast_vec.go b/expression/builtin_cast_vec.go index 4b38c7286953f..f07b4f6d8d4a0 100644 --- a/expression/builtin_cast_vec.go +++ b/expression/builtin_cast_vec.go @@ -44,7 +44,7 @@ func (b *builtinCastIntAsDurationSig) vecEvalDuration(input *chunk.Chunk, result if result.IsNull(i) { continue } - dur, err := types.NumberToDuration(i64s[i], int8(b.tp.Decimal)) + dur, err := types.NumberToDuration(i64s[i], b.tp.Decimal) if err != nil { if types.ErrOverflow.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleOverflow(err, err) @@ -335,7 +335,7 @@ func (b *builtinCastDurationAsIntSig) vecEvalInt(input *chunk.Chunk, result *chu i64s := result.Int64s() var duration types.Duration ds := buf.GoDurations() - fsp := int8(b.args[0].GetType().Decimal) + fsp := b.args[0].GetType().Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -376,7 +376,7 @@ func (b *builtinCastIntAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk. times := result.Times() i64s := buf.Int64s() stmt := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal var tm types.Time for i := 0; i < n; i++ { @@ -483,7 +483,7 @@ func (b *builtinCastJSONAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk result.MergeNulls(buf) times := result.Times() stmtCtx := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -529,7 +529,7 @@ func (b *builtinCastRealAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk times := result.Times() f64s := buf.Float64s() stmt := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if buf.IsNull(i) { continue @@ -608,7 +608,7 @@ func (b *builtinCastDurationAsTimeSig) vecEvalTime(input *chunk.Chunk, result *c ds := buf.GoDurations() times := result.Times() stmtCtx := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -939,7 +939,7 @@ func (b *builtinCastStringAsDurationSig) vecEvalDuration(input *chunk.Chunk, res if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), int8(b.tp.Decimal)) + dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, buf.GetString(i), b.tp.Decimal) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -977,8 +977,8 @@ func (b *builtinCastDurationAsDecimalSig) vecEvalDecimal(input *chunk.Chunk, res var duration types.Duration ds := buf.GoDurations() sc := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.args[0].GetType().Decimal) - if fsp, err = types.CheckFsp(int(fsp)); err != nil { + fsp := b.args[0].GetType().Decimal + if fsp, err = types.CheckFsp(fsp); err != nil { return err } for i := 0; i < n; i++ { @@ -1141,8 +1141,8 @@ func (b *builtinCastDurationAsRealSig) vecEvalReal(input *chunk.Chunk, result *c f64s := result.Float64s() var duration types.Duration - fsp := int8(b.args[0].GetType().Decimal) - if fsp, err = types.CheckFsp(int(fsp)); err != nil { + fsp := b.args[0].GetType().Decimal + if fsp, err = types.CheckFsp(fsp); err != nil { return err } ds := buf.GoDurations() @@ -1213,7 +1213,7 @@ func (b *builtinCastRealAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), int8(b.tp.Decimal)) + dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(f64s[i], 'f', -1, 64), b.tp.Decimal) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1257,7 +1257,7 @@ func (b *builtinCastTimeAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if err != nil { return err } - d, err = d.RoundFrac(int8(b.tp.Decimal), b.ctx.GetSessionVars().Location()) + d, err = d.RoundFrac(b.tp.Decimal, b.ctx.GetSessionVars().Location()) if err != nil { return err } @@ -1286,7 +1286,7 @@ func (b *builtinCastDurationAsDurationSig) vecEvalDuration(input *chunk.Chunk, r continue } dur.Duration = v - rd, err = dur.RoundFrac(int8(b.tp.Decimal), b.ctx.GetSessionVars().Location()) + rd, err = dur.RoundFrac(b.tp.Decimal, b.ctx.GetSessionVars().Location()) if err != nil { return err } @@ -1403,7 +1403,7 @@ func (b *builtinCastDecimalAsTimeSig) vecEvalTime(input *chunk.Chunk, result *ch times := result.Times() decimals := buf.Decimals() stmt := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if buf.IsNull(i) { continue @@ -1473,7 +1473,7 @@ func (b *builtinCastTimeAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chunk times := result.Times() stmt := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -1687,7 +1687,7 @@ func (b *builtinCastStringAsTimeSig) vecEvalTime(input *chunk.Chunk, result *chu result.MergeNulls(buf) times := result.Times() stmtCtx := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -1793,7 +1793,7 @@ func (b *builtinCastDecimalAsDurationSig) vecEvalDuration(input *chunk.Chunk, re if result.IsNull(i) { continue } - dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), int8(b.tp.Decimal)) + dur, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, string(args[i].ToString()), b.tp.Decimal) if err != nil { if types.ErrTruncatedWrongVal.Equal(err) { err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err) @@ -1880,7 +1880,7 @@ func (b *builtinCastJSONAsDurationSig) vecEvalDuration(input *chunk.Chunk, resul if err != nil { return nil } - dur, err = types.ParseDuration(ctx, s, int8(b.tp.Decimal)) + dur, err = types.ParseDuration(ctx, s, b.tp.Decimal) if types.ErrTruncatedWrongVal.Equal(err) { err = ctx.HandleTruncate(err) } diff --git a/expression/builtin_cast_vec_test.go b/expression/builtin_cast_vec_test.go index 5e647be3777dd..20e6395537901 100644 --- a/expression/builtin_cast_vec_test.go +++ b/expression/builtin_cast_vec_test.go @@ -110,7 +110,7 @@ var vecBuiltinCastCases = map[string][]vecExprBenchCase{ type dateTimeGenerWithFsp struct { defaultGener - fsp int8 + fsp int } func (g *dateTimeGenerWithFsp) gen() interface{} { diff --git a/expression/builtin_convert_charset.go b/expression/builtin_convert_charset.go index 02ba0bfd2a0e2..c21cd32c9205b 100644 --- a/expression/builtin_convert_charset.go +++ b/expression/builtin_convert_charset.go @@ -17,6 +17,8 @@ package expression import ( "bytes" "fmt" + "strings" + "unicode" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser/ast" @@ -172,9 +174,10 @@ func (b *builtinInternalFromBinarySig) evalString(row chunk.Row) (res string, is return val, isNull, err } enc := charset.FindEncoding(b.tp.Charset) - ret, err := enc.Transform(nil, hack.Slice(val), charset.OpDecode) + valBytes := hack.Slice(val) + ret, err := enc.Transform(nil, valBytes, charset.OpDecode) if err != nil { - strHex := fmt.Sprintf("%X", val) + strHex := formatInvalidChars(valBytes) err = errCannotConvertString.GenWithStackByArgs(strHex, charset.CharsetBin, b.tp.Charset) } return string(ret), false, err @@ -205,7 +208,7 @@ func (b *builtinInternalFromBinarySig) vecEvalString(input *chunk.Chunk, result str := buf.GetBytes(i) val, err := enc.Transform(encodedBuf, str, charset.OpDecode) if err != nil { - strHex := fmt.Sprintf("%X", str) + strHex := formatInvalidChars(str) return errCannotConvertString.GenWithStackByArgs(strHex, charset.CharsetBin, b.tp.Charset) } result.AppendBytes(val) @@ -334,3 +337,20 @@ func isLegacyCharset(chs string) bool { } return false } + +func formatInvalidChars(src []byte) string { + var sb strings.Builder + const maxBytesToShow = 5 + for i := 0; i < len(src); i++ { + if i > maxBytesToShow { + sb.WriteString("...") + break + } + if src[i] > unicode.MaxASCII { + sb.WriteString(fmt.Sprintf("\\x%X", src[i])) + } else { + sb.Write([]byte{src[i]}) + } + } + return sb.String() +} diff --git a/expression/builtin_encryption.go b/expression/builtin_encryption.go index 134b2c417591c..b66e16fe4f302 100644 --- a/expression/builtin_encryption.go +++ b/expression/builtin_encryption.go @@ -573,17 +573,17 @@ func (b *builtinRandomBytesSig) Clone() builtinFunc { // evalString evals RANDOM_BYTES(len). // See https://dev.mysql.com/doc/refman/5.7/en/encryption-functions.html#function_random-bytes func (b *builtinRandomBytesSig) evalString(row chunk.Row) (string, bool, error) { - len, isNull, err := b.args[0].EvalInt(b.ctx, row) + val, isNull, err := b.args[0].EvalInt(b.ctx, row) if isNull || err != nil { return "", true, err } - if len < 1 || len > 1024 { + if val < 1 || val > 1024 { return "", false, types.ErrOverflow.GenWithStackByArgs("length", "random_bytes") } - buf := make([]byte, len) + buf := make([]byte, val) if n, err := rand.Read(buf); err != nil { return "", true, err - } else if int64(n) != len { + } else if int64(n) != val { return "", false, errors.New("fail to generate random bytes") } return string(buf), false, nil @@ -949,8 +949,7 @@ func (b *builtinUncompressedLengthSig) evalInt(row chunk.Row) (int64, bool, erro sc.AppendWarning(errZlibZData) return 0, false, nil } - len := binary.LittleEndian.Uint32([]byte(payload)[0:4]) - return int64(len), false, nil + return int64(binary.LittleEndian.Uint32([]byte(payload)[0:4])), false, nil } type validatePasswordStrengthFunctionClass struct { diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 22018213ef9e7..2a1d5c7c8e731 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -616,7 +616,35 @@ type charsetFunctionClass struct { } func (c *charsetFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { - return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "CHARSET") + if err := c.verifyArgs(args); err != nil { + return nil, err + } + argsTps := make([]types.EvalType, 0, len(args)) + for _, arg := range args { + argsTps = append(argsTps, arg.GetType().EvalType()) + } + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, argsTps...) + if err != nil { + return nil, err + } + bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() + bf.tp.Flen = 64 + sig := &builtinCharsetSig{bf} + return sig, nil +} + +type builtinCharsetSig struct { + baseBuiltinFunc +} + +func (b *builtinCharsetSig) Clone() builtinFunc { + newSig := &builtinCharsetSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinCharsetSig) evalString(_ chunk.Row) (string, bool, error) { + return b.args[0].GetType().Charset, false, nil } type coercibilityFunctionClass struct { diff --git a/expression/builtin_info_test.go b/expression/builtin_info_test.go index c6b730882c2e9..5089cc3dcfa0d 100644 --- a/expression/builtin_info_test.go +++ b/expression/builtin_info_test.go @@ -192,9 +192,9 @@ func TestCharset(t *testing.T) { ctx := createContext(t) fc := funcs[ast.Charset] f, err := fc.getFunction(ctx, datumsToConstants(types.MakeDatums(nil))) - require.Nil(t, f) - require.Error(t, err) - require.Regexp(t, "FUNCTION CHARSET does not exist$", err.Error()) + require.NotNil(t, f) + require.NoError(t, err) + require.Equal(t, 64, f.getRetTp().Flen) } func TestCoercibility(t *testing.T) { diff --git a/expression/builtin_like_test.go b/expression/builtin_like_test.go index 87f759ec9cf21..15145ea7c9ce2 100644 --- a/expression/builtin_like_test.go +++ b/expression/builtin_like_test.go @@ -98,8 +98,6 @@ func TestRegexp(t *testing.T) { } func TestCILike(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) ctx := createContext(t) tests := []struct { input string diff --git a/expression/builtin_miscellaneous.go b/expression/builtin_miscellaneous.go index 82b43d192c84c..657b377940426 100644 --- a/expression/builtin_miscellaneous.go +++ b/expression/builtin_miscellaneous.go @@ -58,6 +58,7 @@ var ( _ functionClass = &uuidToBinFunctionClass{} _ functionClass = &binToUUIDFunctionClass{} _ functionClass = &isUUIDFunctionClass{} + _ functionClass = &tidbShardFunctionClass{} ) var ( @@ -92,6 +93,11 @@ var ( _ builtinFunc = &builtinNameConstDurationSig{} _ builtinFunc = &builtinNameConstStringSig{} _ builtinFunc = &builtinNameConstJSONSig{} + _ builtinFunc = &builtinTidbShardSig{} +) + +const ( + tidbShardBucketCount = 256 ) type sleepFunctionClass struct { @@ -1304,3 +1310,49 @@ func swapStringUUID(str string) string { copy(buf[18:], str[18:]) return string(buf) } + +type tidbShardFunctionClass struct { + baseFunctionClass +} + +func (c *tidbShardFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, err + } + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, types.ETInt) + if err != nil { + return nil, err + } + + bf.tp.Flen = 4 //64 bit unsigned + bf.tp.Flag |= mysql.UnsignedFlag + types.SetBinChsClnFlag(bf.tp) + + sig := &builtinTidbShardSig{bf} + sig.setPbCode(tipb.ScalarFuncSig_TiDBShard) + return sig, nil +} + +type builtinTidbShardSig struct { + baseBuiltinFunc +} + +func (b *builtinTidbShardSig) Clone() builtinFunc { + newSig := &builtinTidbShardSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +// evalInt evals tidb_shard(int64). +func (b *builtinTidbShardSig) evalInt(row chunk.Row) (int64, bool, error) { + shardKeyInt, isNull, err := b.args[0].EvalInt(b.ctx, row) + if isNull || err != nil { + return 0, true, err + } + var hashed uint64 + if hashed, err = vitess.HashUint64(uint64(shardKeyInt)); err != nil { + return 0, true, err + } + hashed = hashed % tidbShardBucketCount + return int64(hashed), false, nil +} diff --git a/expression/builtin_miscellaneous_test.go b/expression/builtin_miscellaneous_test.go index f125c1b023df3..28e4b42fb0b08 100644 --- a/expression/builtin_miscellaneous_test.go +++ b/expression/builtin_miscellaneous_test.go @@ -577,3 +577,37 @@ func TestBinToUUID(t *testing.T) { _, err := funcs[ast.BinToUUID].getFunction(ctx, []Expression{NewZero()}) require.NoError(t, err) } + +func TestTidbShard(t *testing.T) { + ctx := createContext(t) + + fc := funcs[ast.TiDBShard] + + // tidb_shard(-1) == 81, ...... + args := makeDatums([]int{-1, 0, 1, 9999999999999999}) + res := makeDatums([]int{81, 167, 214, 63}) + for i, arg := range args { + f, err := fc.getFunction(ctx, datumsToConstants([]types.Datum{arg})) + require.NoError(t, err) + d, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + trequire.DatumEqual(t, res[i], d) + } + + // tidb_shard("string") always return 167 + args2 := makeDatums([]string{"abc", "ope", "wopddd"}) + res2 := makeDatums([]int{167}) + for _, arg := range args2 { + f, err := fc.getFunction(ctx, datumsToConstants([]types.Datum{arg})) + require.NoError(t, err) + d, err := evalBuiltinFunc(f, chunk.Row{}) + require.NoError(t, err) + trequire.DatumEqual(t, res2[0], d) + } + + args3 := makeDatums([]int{-1, 0, 1, 9999999999999999}) + { + _, err := fc.getFunction(ctx, datumsToConstants(args3)) + require.Error(t, err) + } +} diff --git a/expression/builtin_other_test.go b/expression/builtin_other_test.go index f786c1b6aeb81..1e865a93cff7d 100644 --- a/expression/builtin_other_test.go +++ b/expression/builtin_other_test.go @@ -298,8 +298,6 @@ func TestInFunc(t *testing.T) { require.NoError(t, err) require.Equalf(t, tc.res, d.GetValue(), "%v", types.MakeDatums(tc.args)) } - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) strD1 := types.NewCollationStringDatum("a", "utf8_general_ci") strD2 := types.NewCollationStringDatum("Á", "utf8_general_ci") fn, err := fc.getFunction(ctx, datumsToConstants([]types.Datum{strD1, strD2})) diff --git a/expression/builtin_string.go b/expression/builtin_string.go index 48068b4d2c17e..4fe2821322447 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -639,6 +639,7 @@ func (c *repeatFunctionClass) getFunction(ctx sessionctx.Context, args []Express return nil, errors.Trace(err) } sig := &builtinRepeatSig{bf, maxAllowedPacket} + sig.setPbCode(tipb.ScalarFuncSig_Repeat) return sig, nil } @@ -3447,6 +3448,7 @@ func (c *fromBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Exp types.SetBinChsClnFlag(bf.tp) sig := &builtinFromBase64Sig{bf, maxAllowedPacket} + sig.setPbCode(tipb.ScalarFuncSig_FromBase64) return sig, nil } @@ -3522,6 +3524,7 @@ func (c *toBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Expre } sig := &builtinToBase64Sig{bf, maxAllowedPacket} + sig.setPbCode(tipb.ScalarFuncSig_ToBase64) return sig, nil } diff --git a/expression/builtin_string_test.go b/expression/builtin_string_test.go index aedb7dc8089ba..0082a4524705c 100644 --- a/expression/builtin_string_test.go +++ b/expression/builtin_string_test.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/testkit/trequire" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -2554,16 +2553,16 @@ func TestWeightString(t *testing.T) { {7, "NONE", 0, nil}, {7.0, "NONE", 0, nil}, {"a", "NONE", 0, "a"}, - {"a ", "NONE", 0, "a "}, + {"a ", "NONE", 0, "a"}, {"中", "NONE", 0, "中"}, - {"中 ", "NONE", 0, "中 "}, + {"中 ", "NONE", 0, "中"}, {nil, "CHAR", 5, nil}, {7, "CHAR", 5, nil}, {7.0, "NONE", 0, nil}, - {"a", "CHAR", 5, "a "}, - {"a ", "CHAR", 5, "a "}, - {"中", "CHAR", 5, "中 "}, - {"中 ", "CHAR", 5, "中 "}, + {"a", "CHAR", 5, "a"}, + {"a ", "CHAR", 5, "a"}, + {"中", "CHAR", 5, "中"}, + {"中 ", "CHAR", 5, "中"}, {nil, "BINARY", 5, nil}, {7, "BINARY", 2, "7\x00"}, {7.0, "NONE", 0, nil}, @@ -2663,8 +2662,6 @@ func TestTranslate(t *testing.T) { func TestCIWeightString(t *testing.T) { ctx := createContext(t) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) type weightStringTest struct { str string diff --git a/expression/builtin_string_vec.go b/expression/builtin_string_vec.go index 5484d46f953e5..8232fa36754b1 100644 --- a/expression/builtin_string_vec.go +++ b/expression/builtin_string_vec.go @@ -43,12 +43,23 @@ func (b *builtinLowerSig) vectorized() bool { } func (b *builtinLowerUTF8Sig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - if err := b.args[0].VecEvalString(b.ctx, input, result); err != nil { + n := input.NumRows() + buf, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } + result.ReserveString(n) enc := charset.FindEncoding(b.args[0].GetType().Charset) - for i := 0; i < input.NumRows(); i++ { - result.SetRaw(i, []byte(enc.ToLower(result.GetString(i)))) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + result.AppendNull() + } else { + result.AppendString(enc.ToLower(buf.GetString(i))) + } } return nil } @@ -144,12 +155,23 @@ func (b *builtinStringIsNullSig) vectorized() bool { } func (b *builtinUpperUTF8Sig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { - if err := b.args[0].VecEvalString(b.ctx, input, result); err != nil { + n := input.NumRows() + buf, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalString(b.ctx, input, buf); err != nil { return err } + result.ReserveString(n) enc := charset.FindEncoding(b.args[0].GetType().Charset) - for i := 0; i < input.NumRows(); i++ { - result.SetRaw(i, []byte(enc.ToUpper(result.GetString(i)))) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + result.AppendNull() + } else { + result.AppendString(enc.ToUpper(buf.GetString(i))) + } } return nil } diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 07942717314cb..0e30348f1eb8c 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -250,7 +250,7 @@ var ( _ builtinFunc = &builtinSubDateDatetimeDecimalSig{} ) -func convertTimeToMysqlTime(t time.Time, fsp int8, roundMode types.RoundMode) (types.Time, error) { +func convertTimeToMysqlTime(t time.Time, fsp int, roundMode types.RoundMode) (types.Time, error) { var tr time.Time var err error if roundMode == types.ModeTruncate { @@ -581,7 +581,7 @@ func (b *builtinDurationStringTimeDiffSig) evalDuration(row chunk.Row) (d types. } sc := b.ctx.GetSessionVars().StmtCtx - rhs, _, isDuration, err := convertStringToDuration(sc, rhsStr, int8(b.tp.Decimal)) + rhs, _, isDuration, err := convertStringToDuration(sc, rhsStr, b.tp.Decimal) if err != nil || !isDuration { return d, true, err } @@ -614,7 +614,7 @@ func (b *builtinStringDurationTimeDiffSig) evalDuration(row chunk.Row) (d types. } sc := b.ctx.GetSessionVars().StmtCtx - lhs, _, isDuration, err := convertStringToDuration(sc, lhsStr, int8(b.tp.Decimal)) + lhs, _, isDuration, err := convertStringToDuration(sc, lhsStr, b.tp.Decimal) if err != nil || !isDuration { return d, true, err } @@ -672,7 +672,7 @@ func (b *builtinTimeStringTimeDiffSig) evalDuration(row chunk.Row) (d types.Dura } sc := b.ctx.GetSessionVars().StmtCtx - _, rhs, isDuration, err := convertStringToDuration(sc, rhsStr, int8(b.tp.Decimal)) + _, rhs, isDuration, err := convertStringToDuration(sc, rhsStr, b.tp.Decimal) if err != nil || isDuration { return d, true, err } @@ -705,7 +705,7 @@ func (b *builtinStringTimeTimeDiffSig) evalDuration(row chunk.Row) (d types.Dura } sc := b.ctx.GetSessionVars().StmtCtx - _, lhs, isDuration, err := convertStringToDuration(sc, lhsStr, int8(b.tp.Decimal)) + _, lhs, isDuration, err := convertStringToDuration(sc, lhsStr, b.tp.Decimal) if err != nil || isDuration { return d, true, err } @@ -738,7 +738,7 @@ func (b *builtinStringStringTimeDiffSig) evalDuration(row chunk.Row) (d types.Du } sc := b.ctx.GetSessionVars().StmtCtx - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal lhsDur, lhsTime, lhsIsDuration, err := convertStringToDuration(sc, lhs, fsp) if err != nil { return d, true, err @@ -780,12 +780,12 @@ func (b *builtinNullTimeDiffSig) evalDuration(row chunk.Row) (d types.Duration, // convertStringToDuration converts string to duration, it return types.Time because in some case // it will converts string to datetime. -func convertStringToDuration(sc *stmtctx.StatementContext, str string, fsp int8) (d types.Duration, t types.Time, +func convertStringToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d types.Duration, t types.Time, isDuration bool, err error) { if n := strings.IndexByte(str, '.'); n >= 0 { lenStrFsp := len(str[n+1:]) - if lenStrFsp <= int(types.MaxFsp) { - fsp = mathutil.MaxInt8(int8(lenStrFsp), fsp) + if lenStrFsp <= types.MaxFsp { + fsp = mathutil.Max(lenStrFsp, fsp) } } return types.StrToDuration(sc, str, fsp) @@ -1680,7 +1680,7 @@ func (c *fromUnixTimeFunctionClass) getFunction(ctx sessionctx.Context, args []E } // Calculate the time fsp. - fsp := int(types.MaxFsp) + fsp := types.MaxFsp if !isArg0Str { if args[0].GetType().Decimal != types.UnspecifiedLength { fsp = mathutil.Min(bf.tp.Decimal, args[0].GetType().Decimal) @@ -1693,8 +1693,8 @@ func (c *fromUnixTimeFunctionClass) getFunction(ctx sessionctx.Context, args []E return sig, nil } -func evalFromUnixTime(ctx sessionctx.Context, fsp int8, unixTimeStamp *types.MyDecimal) (res types.Time, isNull bool, err error) { - // 0 <= unixTimeStamp <= INT32_MAX +func evalFromUnixTime(ctx sessionctx.Context, fsp int, unixTimeStamp *types.MyDecimal) (res types.Time, isNull bool, err error) { + // 0 <= unixTimeStamp <= 32536771199.999999 if unixTimeStamp.IsNegative() { return res, true, nil } @@ -1702,7 +1702,9 @@ func evalFromUnixTime(ctx sessionctx.Context, fsp int8, unixTimeStamp *types.MyD if err != nil && !terror.ErrorEqual(err, types.ErrTruncated) { return res, true, err } - if integralPart > int64(math.MaxInt32) { + // The max integralPart should not be larger than 32536771199. + // Refer to https://dev.mysql.com/doc/relnotes/mysql/8.0/en/news-8-0-28.html + if integralPart > 32536771199 { return res, true, nil } // Split the integral part and fractional part of a decimal timestamp. @@ -1755,7 +1757,7 @@ func (b *builtinFromUnixTime1ArgSig) evalTime(row chunk.Row) (res types.Time, is if err != nil || isNull { return res, isNull, err } - return evalFromUnixTime(b.ctx, int8(b.tp.Decimal), unixTimeStamp) + return evalFromUnixTime(b.ctx, b.tp.Decimal, unixTimeStamp) } type builtinFromUnixTime2ArgSig struct { @@ -1779,7 +1781,7 @@ func (b *builtinFromUnixTime2ArgSig) evalString(row chunk.Row) (res string, isNu if err != nil || isNull { return "", isNull, err } - t, isNull, err := evalFromUnixTime(b.ctx, int8(b.tp.Decimal), unixTimeStamp) + t, isNull, err := evalFromUnixTime(b.ctx, b.tp.Decimal, unixTimeStamp) if isNull || err != nil { return "", isNull, err } @@ -1835,7 +1837,7 @@ type strToDateFunctionClass struct { baseFunctionClass } -func (c *strToDateFunctionClass) getRetTp(ctx sessionctx.Context, arg Expression) (tp byte, fsp int8) { +func (c *strToDateFunctionClass) getRetTp(ctx sessionctx.Context, arg Expression) (tp byte, fsp int) { tp = mysql.TypeDatetime if _, ok := arg.(*Constant); !ok { return tp, types.MaxFsp @@ -1878,7 +1880,7 @@ func (c *strToDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err != nil { return nil, err } - bf.setDecimalAndFlenForDatetime(int(fsp)) + bf.setDecimalAndFlenForDatetime(fsp) sig = &builtinStrToDateDatetimeSig{bf} sig.setPbCode(tipb.ScalarFuncSig_StrToDateDatetime) case mysql.TypeDuration: @@ -1886,7 +1888,7 @@ func (c *strToDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err != nil { return nil, err } - bf.setDecimalAndFlenForTime(int(fsp)) + bf.setDecimalAndFlenForTime(fsp) sig = &builtinStrToDateDurationSig{bf} sig.setPbCode(tipb.ScalarFuncSig_StrToDateDuration) } @@ -1955,7 +1957,7 @@ func (b *builtinStrToDateDatetimeSig) evalTime(row chunk.Row) (types.Time, bool, return types.ZeroTime, true, handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, t.String())) } t.SetType(mysql.TypeDatetime) - t.SetFsp(int8(b.tp.Decimal)) + t.SetFsp(b.tp.Decimal) return t, false, nil } @@ -1987,7 +1989,7 @@ func (b *builtinStrToDateDurationSig) evalDuration(row chunk.Row) (types.Duratio if !succ { return types.Duration{}, true, handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, t.String())) } - t.SetFsp(int8(b.tp.Decimal)) + t.SetFsp(b.tp.Decimal) dur, err := t.ConvertToDuration() return dur, err != nil, err } @@ -2047,7 +2049,7 @@ func (b *builtinSysDateWithFspSig) evalTime(row chunk.Row) (d types.Time, isNull loc := b.ctx.GetSessionVars().Location() now := time.Now().In(loc) - result, err := convertTimeToMysqlTime(now, int8(fsp), types.ModeHalfEven) + result, err := convertTimeToMysqlTime(now, int(fsp), types.ModeHalfEven) if err != nil { return types.ZeroTime, true, err } @@ -2196,7 +2198,7 @@ func (b *builtinCurrentTime1ArgSig) evalDuration(row chunk.Row) (types.Duration, return types.Duration{}, true, err } dur := nowTs.In(tz).Format(types.TimeFSPFormat) - res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int8(fsp)) + res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp)) if err != nil { return types.Duration{}, true, err } @@ -2249,14 +2251,14 @@ func (b *builtinTimeSig) evalDuration(row chunk.Row) (res types.Duration, isNull fsp = len(expr) - idx - 1 } - var tmpFsp int8 + var tmpFsp int if tmpFsp, err = types.CheckFsp(fsp); err != nil { return res, isNull, err } - fsp = int(tmpFsp) + fsp = tmpFsp sc := b.ctx.GetSessionVars().StmtCtx - res, err = types.ParseDuration(sc, expr, int8(fsp)) + res, err = types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2291,7 +2293,7 @@ func (c *timeLiteralFunctionClass) getFunction(ctx sessionctx.Context, args []Ex if err != nil { return nil, err } - bf.setDecimalAndFlenForTime(int(duration.Fsp)) + bf.setDecimalAndFlenForTime(duration.Fsp) sig := &builtinTimeLiteralSig{bf, duration} return sig, nil } @@ -2385,7 +2387,7 @@ func (c *utcTimestampFunctionClass) getFunction(ctx sessionctx.Context, args []E return sig, nil } -func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int8) (types.Time, bool, error) { +func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) { nowTs, err := getStmtTimestamp(ctx) if err != nil { return types.ZeroTime, true, err @@ -2422,7 +2424,7 @@ func (b *builtinUTCTimestampWithArgSig) evalTime(row chunk.Row) (types.Time, boo return types.ZeroTime, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", num) } - result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int8(num)) + result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int(num)) return result, isNull, err } @@ -2439,7 +2441,7 @@ func (b *builtinUTCTimestampWithoutArgSig) Clone() builtinFunc { // evalTime evals UTC_TIMESTAMP(). // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp func (b *builtinUTCTimestampWithoutArgSig) evalTime(row chunk.Row) (types.Time, bool, error) { - result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int8(0)) + result, isNull, err := evalUTCTimestampWithFsp(b.ctx, 0) return result, isNull, err } @@ -2487,7 +2489,7 @@ func GetStmtTimestamp(ctx sessionctx.Context) (time.Time, error) { return tVal.In(tz), nil } -func evalNowWithFsp(ctx sessionctx.Context, fsp int8) (types.Time, bool, error) { +func evalNowWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) { nowTs, err := getStmtTimestamp(ctx) if err != nil { return types.ZeroTime, true, err @@ -2545,7 +2547,7 @@ func (b *builtinNowWithArgSig) evalTime(row chunk.Row) (types.Time, bool, error) return types.ZeroTime, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp) } - result, isNull, err := evalNowWithFsp(b.ctx, int8(fsp)) + result, isNull, err := evalNowWithFsp(b.ctx, int(fsp)) return result, isNull, err } @@ -2562,7 +2564,7 @@ func (b *builtinNowWithoutArgSig) Clone() builtinFunc { // evalTime evals NOW() // see: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_now func (b *builtinNowWithoutArgSig) evalTime(row chunk.Row) (types.Time, bool, error) { - result, isNull, err := evalNowWithFsp(b.ctx, int8(0)) + result, isNull, err := evalNowWithFsp(b.ctx, 0) return result, isNull, err } @@ -2673,7 +2675,7 @@ func (b *builtinExtractDatetimeFromStringSig) evalInt(row chunk.Row) (int64, boo } } if dt.IsZero() { - dt.SetFsp(int8(b.args[1].GetType().Decimal)) + dt.SetFsp(b.args[1].GetType().Decimal) if b.ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() { isNull, err := handleInvalidZeroTime(b.ctx, dt) return 0, isNull, err @@ -3292,12 +3294,12 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres switch unit { // If the unit has micro second, then the fsp must be the MaxFsp. case "MICROSECOND", "SECOND_MICROSECOND", "MINUTE_MICROSECOND", "HOUR_MICROSECOND", "DAY_MICROSECOND": - internalFsp = int(types.MaxFsp) + internalFsp = types.MaxFsp // If the unit is second, the fsp is related with the arg[1]'s. case "SECOND": - internalFsp = int(types.MaxFsp) + internalFsp = types.MaxFsp if intervalEvalTp != types.ETString { - internalFsp = mathutil.Min(args[1].GetType().Decimal, int(types.MaxFsp)) + internalFsp = mathutil.Min(args[1].GetType().Decimal, types.MaxFsp) } // Otherwise, the fsp should be 0. } @@ -4081,12 +4083,12 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres switch unit { // If the unit has micro second, then the fsp must be the MaxFsp. case "MICROSECOND", "SECOND_MICROSECOND", "MINUTE_MICROSECOND", "HOUR_MICROSECOND", "DAY_MICROSECOND": - internalFsp = int(types.MaxFsp) + internalFsp = types.MaxFsp // If the unit is second, the fsp is related with the arg[1]'s. case "SECOND": - internalFsp = int(types.MaxFsp) + internalFsp = types.MaxFsp if intervalEvalTp != types.ETString { - internalFsp = mathutil.Min(args[1].GetType().Decimal, int(types.MaxFsp)) + internalFsp = mathutil.Min(args[1].GetType().Decimal, types.MaxFsp) } // Otherwise, the fsp should be 0. } @@ -5247,7 +5249,7 @@ func (c *timestampLiteralFunctionClass) getFunction(ctx sessionctx.Context, args if err != nil { return nil, err } - bf.setDecimalAndFlenForDatetime(int(tm.Fsp())) + bf.setDecimalAndFlenForDatetime(tm.Fsp()) sig := &builtinTimestampLiteralSig{bf, tm} return sig, nil } @@ -5272,7 +5274,7 @@ func (b *builtinTimestampLiteralSig) evalTime(row chunk.Row) (types.Time, bool, // getFsp4TimeAddSub is used to in function 'ADDTIME' and 'SUBTIME' to evaluate `fsp` for the // second parameter. It's used only if the second parameter is of string type. It's different // from getFsp in that the result of getFsp4TimeAddSub is either 6 or 0. -func getFsp4TimeAddSub(s string) int8 { +func getFsp4TimeAddSub(s string) int { if len(s)-strings.Index(s, ".")-1 == len(s) { return types.MinFsp } @@ -5320,9 +5322,9 @@ func getBf4TimeAddSub(ctx sessionctx.Context, funcName string, args []Expression } switch retTp { case types.ETDatetime: - bf.setDecimalAndFlenForDatetime(mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), int(types.MaxFsp))) + bf.setDecimalAndFlenForDatetime(mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), types.MaxFsp)) case types.ETDuration: - bf.setDecimalAndFlenForTime(mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), int(types.MaxFsp))) + bf.setDecimalAndFlenForTime(mathutil.Min(mathutil.Max(arg0Dec, arg1Dec), types.MaxFsp)) case types.ETString: bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeString, mysql.MaxDatetimeWidthWithFsp, types.UnspecifiedLength } @@ -5837,7 +5839,7 @@ type convertTzFunctionClass struct { } func (c *convertTzFunctionClass) getDecimal(ctx sessionctx.Context, arg Expression) int { - decimal := int(types.MaxFsp) + decimal := types.MaxFsp if dt, isConstant := arg.(*Constant); isConstant { switch arg.GetType().EvalType() { case types.ETInt: @@ -5851,11 +5853,11 @@ func (c *convertTzFunctionClass) getDecimal(ctx sessionctx.Context, arg Expressi } } } - if decimal > int(types.MaxFsp) { - return int(types.MaxFsp) + if decimal > types.MaxFsp { + return types.MaxFsp } - if decimal < int(types.MinFsp) { - return int(types.MinFsp) + if decimal < types.MinFsp { + return types.MinFsp } return decimal } @@ -5957,7 +5959,7 @@ func (b *builtinConvertTzSig) convertTz(dt types.Time, fromTzStr, toTzStr string } } - return types.NewTime(types.FromGoTime(t.In(toTz)), mysql.TypeDatetime, int8(b.tp.Decimal)), false, nil + return types.NewTime(types.FromGoTime(t.In(toTz)), mysql.TypeDatetime, b.tp.Decimal), false, nil } type makeDateFunctionClass struct { @@ -6084,7 +6086,7 @@ func (b *builtinMakeTimeSig) makeTime(hour int64, minute int64, second float64, second = 59 } fsp := b.tp.Decimal - return types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), int8(fsp)) + return types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%02d:%02d:%v", hour, minute, second), fsp) } // evalDuration evals a builtinMakeTimeIntSig. @@ -6313,10 +6315,10 @@ func (c *secToTimeFunctionClass) getFunction(ctx sessionctx.Context, args []Expr } else { retFsp = argType.Decimal } - if retFsp > int(types.MaxFsp) || retFsp == int(types.UnspecifiedFsp) { - retFsp = int(types.MaxFsp) - } else if retFsp < int(types.MinFsp) { - retFsp = int(types.MinFsp) + if retFsp > types.MaxFsp || retFsp == types.UnspecifiedFsp { + retFsp = types.MaxFsp + } else if retFsp < types.MinFsp { + retFsp = types.MinFsp } bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDuration, types.ETReal) if err != nil { @@ -6378,7 +6380,7 @@ func (b *builtinSecToTimeSig) evalDuration(row chunk.Row) (types.Duration, bool, secondDemical = float64(second) + demical var dur types.Duration - dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), int8(b.tp.Decimal)) + dur, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.Decimal) if err != nil { return types.Duration{}, err != nil, err } @@ -7174,7 +7176,7 @@ func (b *builtinUTCTimeWithArgSig) evalDuration(row chunk.Row) (types.Duration, if err != nil { return types.Duration{}, true, err } - v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int8(fsp)) + v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp)) return v, false, err } @@ -7232,10 +7234,10 @@ func getExpressionFsp(ctx sessionctx.Context, expression Expression) (int, error if isNil || err != nil { return 0, err } - return int(types.GetFsp(str)), nil + return types.GetFsp(str), nil } warpExpr := WrapWithCastAsTime(ctx, expression, types.NewFieldType(mysql.TypeDatetime)) - return mathutil.Min(warpExpr.GetType().Decimal, int(types.MaxFsp)), nil + return mathutil.Min(warpExpr.GetType().Decimal, types.MaxFsp), nil } // tidbParseTsoFunctionClass extracts physical time from a tso @@ -7253,7 +7255,7 @@ func (c *tidbParseTsoFunctionClass) getFunction(ctx sessionctx.Context, args []E return nil, err } - bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, int(types.DefaultFsp) + bf.tp.Tp, bf.tp.Flen, bf.tp.Decimal = mysql.TypeDate, mysql.MaxDateWidth, types.DefaultFsp sig := &builtinTidbParseTsoSig{bf} return sig, nil } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 5bedae12c0e1c..6412a2a52a644 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -948,7 +948,7 @@ func TestAddTimeSig(t *testing.T) { require.NoError(t, err) res, _, err := du.add(ctx, now, "1", "MICROSECOND") require.NoError(t, err) - require.Equal(t, int8(6), res.Fsp()) + require.Equal(t, 6, res.Fsp()) tbl = []struct { Input string @@ -1180,13 +1180,13 @@ func TestSysDate(t *testing.T) { require.Error(t, err) } -func convertToTimeWithFsp(sc *stmtctx.StatementContext, arg types.Datum, tp byte, fsp int8) (d types.Datum, err error) { +func convertToTimeWithFsp(sc *stmtctx.StatementContext, arg types.Datum, tp byte, fsp int) (d types.Datum, err error) { if fsp > types.MaxFsp { fsp = types.MaxFsp } f := types.NewFieldType(tp) - f.Decimal = int(fsp) + f.Decimal = fsp d, err = arg.ConvertTo(sc, f) if err != nil { @@ -1604,7 +1604,7 @@ func TestTimeDiff(t *testing.T) { args []interface{} expectStr string isNil bool - fsp int8 + fsp int flen int getWarning bool }{ @@ -1995,7 +1995,7 @@ func TestDateArithFuncs(t *testing.T) { testDurations := []struct { fc functionClass dur string - fsp int8 + fsp int unit string format interface{} expected string diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 896cea3252584..2afcb67059113 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -138,7 +138,7 @@ func (b *builtinFromUnixTime2ArgSig) vecEvalString(input *chunk.Chunk, result *c result.ReserveString(n) ds := buf1.Decimals() - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if buf1.IsNull(i) || buf2.IsNull(i) { result.AppendNull() @@ -421,7 +421,7 @@ func (b *builtinUTCTimeWithArgSig) vecEvalDuration(input *chunk.Chunk, result *c if fsp < int64(types.MinFsp) { return errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp) } - res, err := types.ParseDuration(stmtCtx, utc, int8(fsp)) + res, err := types.ParseDuration(stmtCtx, utc, int(fsp)) if err != nil { return err } @@ -554,7 +554,7 @@ func (b *builtinNowWithArgSig) vecEvalTime(input *chunk.Chunk, result *chunk.Col fsps := bufFsp.Int64s() for i := 0; i < n; i++ { - fsp := int8(0) + fsp := 0 if !bufFsp.IsNull(i) { if fsps[i] > int64(types.MaxFsp) { return errors.Errorf("Too-big precision %v specified for 'now'. Maximum is %v.", fsps[i], types.MaxFsp) @@ -562,7 +562,7 @@ func (b *builtinNowWithArgSig) vecEvalTime(input *chunk.Chunk, result *chunk.Col if fsps[i] < int64(types.MinFsp) { return errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsps[i]) } - fsp = int8(fsps[i]) + fsp = int(fsps[i]) } t, isNull, err := evalNowWithFsp(b.ctx, fsp) @@ -775,7 +775,7 @@ func (b *builtinSysDateWithFspSig) vecEvalTime(input *chunk.Chunk, result *chunk if result.IsNull(i) { continue } - t, err := convertTimeToMysqlTime(now, int8(ds[i]), types.ModeHalfEven) + t, err := convertTimeToMysqlTime(now, int(ds[i]), types.ModeHalfEven) if err != nil { return err } @@ -1090,7 +1090,7 @@ func (b *builtinExtractDurationSig) vecEvalInt(input *chunk.Chunk, result *chunk i64s := result.Int64s() durIs := dur.GoDurations() var duration types.Duration - duration.Fsp = int8(b.args[1].GetType().Decimal) + duration.Fsp = b.args[1].GetType().Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -1154,7 +1154,7 @@ func (b *builtinStrToDateDurationSig) vecEvalDuration(input *chunk.Chunk, result result.SetNull(i, true) continue } - t.SetFsp(int8(b.tp.Decimal)) + t.SetFsp(b.tp.Decimal) dur, err := t.ConvertToDuration() if err != nil { return err @@ -1225,7 +1225,7 @@ func (b *builtinMinuteSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) if result.IsNull(i) { continue } - i64s[i] = int64(buf.GetDuration(i, int(types.UnspecifiedFsp)).Minute()) + i64s[i] = int64(buf.GetDuration(i, types.UnspecifiedFsp).Minute()) } return nil } @@ -1252,7 +1252,7 @@ func (b *builtinSecondSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) if result.IsNull(i) { continue } - i64s[i] = int64(buf.GetDuration(i, int(types.UnspecifiedFsp)).Second()) + i64s[i] = int64(buf.GetDuration(i, types.UnspecifiedFsp).Second()) } return nil } @@ -1263,7 +1263,7 @@ func (b *builtinNowWithoutArgSig) vectorized() bool { func (b *builtinNowWithoutArgSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { n := input.NumRows() - nowTs, isNull, err := evalNowWithFsp(b.ctx, int8(0)) + nowTs, isNull, err := evalNowWithFsp(b.ctx, 0) if err != nil { return err } @@ -1422,7 +1422,7 @@ func (b *builtinUTCTimestampWithArgSig) vecEvalTime(input *chunk.Chunk, result * if fsp < int64(types.MinFsp) { return errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp) } - res, isNull, err := evalUTCTimestampWithFsp(b.ctx, int8(fsp)) + res, isNull, err := evalUTCTimestampWithFsp(b.ctx, int(fsp)) if err != nil { return err } @@ -1501,7 +1501,7 @@ func (b *builtinStrToDateDatetimeSig) vecEvalTime(input *chunk.Chunk, result *ch times := result.Times() sc := b.ctx.GetSessionVars().StmtCtx hasNoZeroDateMode := b.ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { @@ -1954,7 +1954,7 @@ func (b *builtinSecToTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk. second = seconds % 60 } secondDemical := float64(second) + demical - duration, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), int8(b.tp.Decimal)) + duration, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, fmt.Sprintf("%s%02d:%02d:%s", negative, hour, minute, strconv.FormatFloat(secondDemical, 'f', -1, 64)), b.tp.Decimal) if err != nil { return err } @@ -2180,7 +2180,7 @@ func (b *builtinFromUnixTime1ArgSig) vecEvalTime(input *chunk.Chunk, result *chu result.MergeNulls(buf) ts := result.Times() ds := buf.Decimals() - fsp := int8(b.tp.Decimal) + fsp := b.tp.Decimal for i := 0; i < n; i++ { if result.IsNull(i) { continue @@ -2420,13 +2420,13 @@ func (b *builtinTimeSig) vecEvalDuration(input *chunk.Chunk, result *chunk.Colum fsp = len(expr) - idx - 1 } - var tmpFsp int8 + var tmpFsp int if tmpFsp, err = types.CheckFsp(fsp); err != nil { return err } - fsp = int(tmpFsp) + fsp = tmpFsp - res, err := types.ParseDuration(sc, expr, int8(fsp)) + res, err := types.ParseDuration(sc, expr, fsp) if types.ErrTruncatedWrongVal.Equal(err) { err = sc.HandleTruncate(err) } @@ -2572,7 +2572,7 @@ func (b *builtinCurrentTime1ArgSig) vecEvalDuration(input *chunk.Chunk, result * result.ResizeGoDuration(n, false) durations := result.GoDurations() for i := 0; i < n; i++ { - res, err := types.ParseDuration(stmtCtx, dur, int8(i64s[i])) + res, err := types.ParseDuration(stmtCtx, dur, int(i64s[i])) if err != nil { return err } diff --git a/expression/builtin_time_vec_generated.go b/expression/builtin_time_vec_generated.go index 52fc31a902c57..4081890fb23c3 100644 --- a/expression/builtin_time_vec_generated.go +++ b/expression/builtin_time_vec_generated.go @@ -316,7 +316,7 @@ func (b *builtinAddStringAndDurationSig) vecEvalString(input *chunk.Chunk, resul // calculate sc := b.ctx.GetSessionVars().StmtCtx - fsp1 := int8(b.args[1].GetType().Decimal) + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} var output string var isNull bool @@ -498,8 +498,8 @@ func (b *builtinAddDateAndDurationSig) vecEvalString(input *chunk.Chunk, result // calculate - fsp0 := int8(b.args[0].GetType().Decimal) - fsp1 := int8(b.args[1].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) @@ -576,7 +576,7 @@ func (b *builtinAddDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c return err } - fsp0 := int8(b.args[0].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) @@ -940,7 +940,7 @@ func (b *builtinSubStringAndDurationSig) vecEvalString(input *chunk.Chunk, resul // calculate sc := b.ctx.GetSessionVars().StmtCtx - fsp1 := int8(b.args[1].GetType().Decimal) + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} var output string var isNull bool @@ -1122,8 +1122,8 @@ func (b *builtinSubDateAndDurationSig) vecEvalString(input *chunk.Chunk, result // calculate - fsp0 := int8(b.args[0].GetType().Decimal) - fsp1 := int8(b.args[1].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Sub(arg1Duration) @@ -1200,7 +1200,7 @@ func (b *builtinSubDateAndStringSig) vecEvalString(input *chunk.Chunk, result *c return err } - fsp0 := int8(b.args[0].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Sub(arg1Duration) @@ -1301,7 +1301,7 @@ func (b *builtinTimeStringTimeDiffSig) vecEvalDuration(input *chunk.Chunk, resul continue } lhsTime := arg0[i] - _, rhsTime, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), int8(b.tp.Decimal)) + _, rhsTime, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -1356,7 +1356,7 @@ func (b *builtinDurationStringTimeDiffSig) vecEvalDuration(input *chunk.Chunk, r continue } lhs.Duration = arg0[i] - rhsDur, _, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), int8(b.tp.Decimal)) + rhsDur, _, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -1460,7 +1460,7 @@ func (b *builtinStringTimeTimeDiffSig) vecEvalDuration(input *chunk.Chunk, resul if result.IsNull(i) { continue } - _, lhsTime, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), int8(b.tp.Decimal)) + _, lhsTime, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -1515,7 +1515,7 @@ func (b *builtinStringDurationTimeDiffSig) vecEvalDuration(input *chunk.Chunk, r if result.IsNull(i) { continue } - lhsDur, _, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), int8(b.tp.Decimal)) + lhsDur, _, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -1571,11 +1571,11 @@ func (b *builtinStringStringTimeDiffSig) vecEvalDuration(input *chunk.Chunk, res if result.IsNull(i) { continue } - lhsDur, lhsTime, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), int8(b.tp.Decimal)) + lhsDur, lhsTime, lhsIsDuration, err := convertStringToDuration(stmtCtx, buf0.GetString(i), b.tp.Decimal) if err != nil { return err } - rhsDur, rhsTime, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), int8(b.tp.Decimal)) + rhsDur, rhsTime, rhsIsDuration, err := convertStringToDuration(stmtCtx, buf1.GetString(i), b.tp.Decimal) if err != nil { return err } diff --git a/expression/builtin_time_vec_generated_test.go b/expression/builtin_time_vec_generated_test.go index a6a2fb0c73b3f..e41fadeb461d7 100644 --- a/expression/builtin_time_vec_generated_test.go +++ b/expression/builtin_time_vec_generated_test.go @@ -34,7 +34,7 @@ func (g gener) gen() interface{} { if _, ok := result.(string); ok { dg := newDefaultGener(0, types.ETDuration) d := dg.gen().(types.Duration) - if int8(d.Duration)%2 == 0 { + if d.Duration%2 == 0 { d.Fsp = 0 } else { d.Fsp = 1 diff --git a/expression/collation.go b/expression/collation.go index 40f3104dd48ae..9febc5b423b16 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -175,7 +175,12 @@ func deriveCoercibilityForColumn(c *Column) Coercibility { if c.RetType.Tp == mysql.TypeNull { return CoercibilityIgnorable } - if c.RetType.EvalType() != types.ETString { + + switch c.RetType.EvalType() { + case types.ETJson: + case types.ETString: + return CoercibilityImplicit + default: return CoercibilityNumeric } return CoercibilityImplicit @@ -366,15 +371,24 @@ func inferCollation(exprs ...Expression) *ExprCollation { repertoire := exprs[0].Repertoire() coercibility := exprs[0].Coercibility() dstCharset, dstCollation := exprs[0].GetType().Charset, exprs[0].GetType().Collate + if exprs[0].GetType().EvalType() == types.ETJson { + dstCharset, dstCollation = charset.CharsetUTF8MB4, charset.CollationUTF8MB4 + } unknownCS := false // Aggregate arguments one by one, agg(a, b, c) := agg(agg(a, b), c). for _, arg := range exprs[1:] { + argCharset, argCollation := arg.GetType().Charset, arg.GetType().Collate + // The collation of JSON is always utf8mb4_bin in builtin-func which is same as MySQL + // see details https://github.com/pingcap/tidb/issues/31320#issuecomment-1010599311 + if arg.GetType().EvalType() == types.ETJson { + argCharset, argCollation = charset.CharsetUTF8MB4, charset.CollationUTF8MB4 + } // If one of the arguments is binary charset, we allow it can be used with other charsets. // If they have the same coercibility, let the binary charset one to be the winner because binary has more precedence. - if dstCollation == charset.CollationBin || arg.GetType().Collate == charset.CollationBin { - if coercibility > arg.Coercibility() || (coercibility == arg.Coercibility() && arg.GetType().Collate == charset.CollationBin) { - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + if dstCollation == charset.CollationBin || argCollation == charset.CollationBin { + if coercibility > arg.Coercibility() || (coercibility == arg.Coercibility() && argCollation == charset.CollationBin) { + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation } repertoire |= arg.Repertoire() continue @@ -387,7 +401,7 @@ func inferCollation(exprs ...Expression) *ExprCollation { // 4. constant value is allowed because we can eval and convert it directly. // If we can not aggregate these two collations, we will get CoercibilityNone and wait for an explicit COLLATE clause, if // there is no explicit COLLATE clause, we will get an error. - if dstCharset != arg.GetType().Charset { + if dstCharset != argCharset { switch { case coercibility < arg.Coercibility(): if arg.Repertoire() == ASCII || arg.Coercibility() >= CoercibilitySysconst || isUnicodeCollation(dstCharset) { @@ -395,15 +409,15 @@ func inferCollation(exprs ...Expression) *ExprCollation { continue } case coercibility == arg.Coercibility(): - if (isUnicodeCollation(dstCharset) && !isUnicodeCollation(arg.GetType().Charset)) || (dstCharset == charset.CharsetUTF8MB4 && arg.GetType().Charset == charset.CharsetUTF8) { + if (isUnicodeCollation(dstCharset) && !isUnicodeCollation(argCharset)) || (dstCharset == charset.CharsetUTF8MB4 && argCharset == charset.CharsetUTF8) { repertoire |= arg.Repertoire() continue - } else if (isUnicodeCollation(arg.GetType().Charset) && !isUnicodeCollation(dstCharset)) || (arg.GetType().Charset == charset.CharsetUTF8MB4 && dstCharset == charset.CharsetUTF8) { - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + } else if (isUnicodeCollation(argCharset) && !isUnicodeCollation(dstCharset)) || (argCharset == charset.CharsetUTF8MB4 && dstCharset == charset.CharsetUTF8) { + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation repertoire |= arg.Repertoire() continue } else if repertoire == ASCII && arg.Repertoire() != ASCII { - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation repertoire |= arg.Repertoire() continue } else if repertoire != ASCII && arg.Repertoire() == ASCII { @@ -411,8 +425,8 @@ func inferCollation(exprs ...Expression) *ExprCollation { continue } case coercibility > arg.Coercibility(): - if repertoire == ASCII || coercibility >= CoercibilitySysconst || isUnicodeCollation(arg.GetType().Charset) { - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + if repertoire == ASCII || coercibility >= CoercibilitySysconst || isUnicodeCollation(argCharset) { + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation repertoire |= arg.Repertoire() continue } @@ -427,17 +441,17 @@ func inferCollation(exprs ...Expression) *ExprCollation { // derive to CoercibilityNone and _bin collation. switch { case coercibility == arg.Coercibility(): - if dstCollation == arg.GetType().Collate { + if dstCollation == argCollation { } else if coercibility == CoercibilityExplicit { return nil } else if isBinCollation(dstCollation) { - } else if isBinCollation(arg.GetType().Collate) { - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + } else if isBinCollation(argCollation) { + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation } else { - coercibility, dstCollation, dstCharset = CoercibilityNone, getBinCollation(arg.GetType().Charset), arg.GetType().Charset + coercibility, dstCharset, dstCollation = CoercibilityNone, argCharset, getBinCollation(argCharset) } case coercibility > arg.Coercibility(): - coercibility, dstCharset, dstCollation = arg.Coercibility(), arg.GetType().Charset, arg.GetType().Collate + coercibility, dstCharset, dstCollation = arg.Coercibility(), argCharset, argCollation } repertoire |= arg.Repertoire() } diff --git a/expression/collation_test.go b/expression/collation_test.go index 9e11c66089d12..335c55368ec5c 100644 --- a/expression/collation_test.go +++ b/expression/collation_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/stretchr/testify/require" ) @@ -252,6 +251,11 @@ func newColString(chs, coll string) *Column { return column } +func newColJSON() *Column { + column := &Column{RetType: &types.FieldType{Tp: mysql.TypeJSON, Charset: charset.CharsetBinary, Collate: charset.CollationBin}} + return column +} + func newConstInt(coercibility Coercibility) *Constant { constant := &Constant{RetType: &types.FieldType{Tp: mysql.TypeLong, Charset: charset.CharsetBin, Collate: charset.CollationBin}, Value: types.NewDatum(1)} constant.SetCoercibility(coercibility) @@ -506,6 +510,46 @@ func TestDeriveCollation(t *testing.T) { false, &ExprCollation{CoercibilityImplicit, UNICODE, charset.CharsetUTF8MB4, charset.CollationUTF8MB4}, }, + { + []string{ + ast.ExportSet, ast.Elt, ast.MakeSet, + }, + []Expression{ + newColInt(CoercibilityExplicit), + newColJSON(), + newColString(charset.CharsetUTF8MB4, "utf8mb4_unicode_ci"), + }, + []types.EvalType{types.ETInt, types.ETJson}, + types.ETString, + false, + &ExprCollation{CoercibilityImplicit, UNICODE, charset.CharsetUTF8MB4, charset.CollationUTF8MB4}, + }, + { + []string{ + ast.Concat, ast.ConcatWS, ast.Coalesce, ast.Greatest, ast.Least, + }, + []Expression{ + newColString(charset.CharsetGBK, charset.CollationGBKBin), + newColJSON(), + }, + []types.EvalType{types.ETString, types.ETJson}, + types.ETString, + false, + &ExprCollation{CoercibilityImplicit, UNICODE, charset.CharsetUTF8MB4, charset.CollationUTF8MB4}, + }, + { + []string{ + ast.Concat, ast.ConcatWS, ast.Coalesce, ast.Greatest, ast.Least, + }, + []Expression{ + newColJSON(), + newColString(charset.CharsetBinary, charset.CharsetBinary), + }, + []types.EvalType{types.ETJson, types.ETString}, + types.ETString, + false, + &ExprCollation{CoercibilityImplicit, UNICODE, charset.CharsetBinary, charset.CharsetBinary}, + }, { []string{ ast.Concat, ast.ConcatWS, ast.Coalesce, ast.In, ast.Greatest, ast.Least, @@ -532,6 +576,18 @@ func TestDeriveCollation(t *testing.T) { false, &ExprCollation{CoercibilityCoercible, ASCII, charset.CharsetUTF8MB4, charset.CollationUTF8MB4}, }, + { + []string{ + ast.Lower, ast.Lcase, ast.Reverse, ast.Upper, ast.Ucase, ast.Quote, + }, + []Expression{ + newColJSON(), + }, + []types.EvalType{types.ETString}, + types.ETString, + false, + &ExprCollation{CoercibilityImplicit, UNICODE, charset.CharsetUTF8MB4, charset.CollationUTF8MB4}, + }, { []string{ ast.If, @@ -637,9 +693,6 @@ func TestDeriveCollation(t *testing.T) { } func TestCompareString(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - require.Equal(t, 0, types.CompareString("a", "A", "utf8_general_ci")) require.Equal(t, 0, types.CompareString("À", "A", "utf8_general_ci")) require.Equal(t, 0, types.CompareString("😜", "😃", "utf8_general_ci")) diff --git a/expression/column.go b/expression/column.go index 378eae5bd674a..9eda8d9e11089 100644 --- a/expression/column.go +++ b/expression/column.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -552,8 +553,8 @@ func ColInfo2Col(cols []*Column, col *model.ColumnInfo) *Column { return nil } -// indexCol2Col finds the corresponding column of the IndexColumn in a column slice. -func indexCol2Col(colInfos []*model.ColumnInfo, cols []*Column, col *model.IndexColumn) *Column { +// IndexCol2Col finds the corresponding column of the IndexColumn in a column slice. +func IndexCol2Col(colInfos []*model.ColumnInfo, cols []*Column, col *model.IndexColumn) *Column { for i, info := range colInfos { if info.Name.L == col.Name.L { if col.Length > 0 && info.FieldType.Flen > col.Length { @@ -576,7 +577,7 @@ func IndexInfo2PrefixCols(colInfos []*model.ColumnInfo, cols []*Column, index *m retCols := make([]*Column, 0, len(index.Columns)) lengths := make([]int, 0, len(index.Columns)) for _, c := range index.Columns { - col := indexCol2Col(colInfos, cols, c) + col := IndexCol2Col(colInfos, cols, c) if col == nil { return retCols, lengths } @@ -598,7 +599,7 @@ func IndexInfo2Cols(colInfos []*model.ColumnInfo, cols []*Column, index *model.I retCols := make([]*Column, 0, len(index.Columns)) lens := make([]int, 0, len(index.Columns)) for _, c := range index.Columns { - col := indexCol2Col(colInfos, cols, c) + col := IndexCol2Col(colInfos, cols, c) if col == nil { retCols = append(retCols, col) lens = append(lens, types.UnspecifiedLength) @@ -662,13 +663,17 @@ func (col *Column) Coercibility() Coercibility { // Repertoire returns the repertoire value which is used to check collations. func (col *Column) Repertoire() Repertoire { - if col.RetType.EvalType() != types.ETString { - return ASCII - } - if col.RetType.Charset == charset.CharsetASCII { + switch col.RetType.EvalType() { + case types.ETJson: + return UNICODE + case types.ETString: + if col.RetType.Charset == charset.CharsetASCII { + return ASCII + } + return UNICODE + default: return ASCII } - return UNICODE } // SortColumns sort columns based on UniqueID. @@ -680,3 +685,31 @@ func SortColumns(cols []*Column) []*Column { }) return sorted } + +// InColumnArray check whether the col is in the cols array +func (col *Column) InColumnArray(cols []*Column) bool { + for _, c := range cols { + if col.Equal(nil, c) { + return true + } + } + return false +} + +// GcColumnExprIsTidbShard check whether the expression is tidb_shard() +func GcColumnExprIsTidbShard(virtualExpr Expression) bool { + if virtualExpr == nil { + return false + } + + f, ok := virtualExpr.(*ScalarFunction) + if !ok { + return false + } + + if f.FuncName.L != ast.TiDBShard { + return false + } + + return true +} diff --git a/expression/column_test.go b/expression/column_test.go index f6fa04e784110..631081a1ebd3a 100644 --- a/expression/column_test.go +++ b/expression/column_test.go @@ -18,6 +18,7 @@ import ( "fmt" "testing" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" @@ -228,3 +229,37 @@ func TestColHybird(t *testing.T) { require.Equal(t, result.GetString(i), v) } } + +func TestInColumnArray(t *testing.T) { + // normal case, col is in column array + col0, col1 := &Column{ID: 0, UniqueID: 0}, &Column{ID: 1, UniqueID: 1} + cols := []*Column{col0, col1} + require.True(t, col0.InColumnArray(cols)) + + // abnormal case, col is not in column array + require.False(t, col0.InColumnArray([]*Column{col1})) + + // abnormal case, input is nil + require.False(t, col0.InColumnArray(nil)) +} + +func TestGcColumnExprIsTidbShard(t *testing.T) { + ctx := mock.NewContext() + + // abnormal case + // nil, not tidb_shard + require.False(t, GcColumnExprIsTidbShard(nil)) + + // `a = 1`, not tidb_shard + ft := types.NewFieldType(mysql.TypeLonglong) + col := &Column{RetType: ft, Index: 0} + d1 := types.NewDatum(1) + con := &Constant{Value: d1, RetType: ft} + expr := NewFunctionInternal(ctx, ast.EQ, ft, col, con) + require.False(t, GcColumnExprIsTidbShard(expr)) + + // normal case + // tidb_shard(a) = 1 + shardExpr := NewFunctionInternal(ctx, ast.TiDBShard, ft, col) + require.True(t, GcColumnExprIsTidbShard(shardExpr)) +} diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 655031370a10c..3365222e13b1a 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -1180,7 +1180,7 @@ func convertTime(data []byte, ftPB *tipb.FieldType, tz *time.Location) (*Constan } var t types.Time t.SetType(ft.Tp) - t.SetFsp(int8(ft.Decimal)) + t.SetFsp(ft.Decimal) err = t.FromPackedUint(v) if err != nil { return nil, err diff --git a/expression/distsql_builtin_test.go b/expression/distsql_builtin_test.go index 5bf90f09bc3cb..26f47169e9098 100644 --- a/expression/distsql_builtin_test.go +++ b/expression/distsql_builtin_test.go @@ -793,6 +793,7 @@ func TestEval(t *testing.T) { } func TestPBToExprWithNewCollation(t *testing.T) { + collate.SetNewCollationEnabledForTest(false) sc := new(stmtctx.StatementContext) fieldTps := make([]*types.FieldType, 1) @@ -829,7 +830,6 @@ func TestPBToExprWithNewCollation(t *testing.T) { } collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) for _, cs := range cases { ft := types.NewFieldType(mysql.TypeString) @@ -963,7 +963,7 @@ func newIntFieldType() *types.FieldType { func newDurFieldType() *types.FieldType { return &types.FieldType{ Tp: mysql.TypeDuration, - Decimal: int(types.DefaultFsp), + Decimal: types.DefaultFsp, } } diff --git a/expression/evaluator_test.go b/expression/evaluator_test.go index 50a27f21f8d48..ab9bc5930bfec 100644 --- a/expression/evaluator_test.go +++ b/expression/evaluator_test.go @@ -19,7 +19,6 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/mysql" @@ -30,12 +29,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestT(t *testing.T) { - CustomVerboseFlag = true - *CustomParallelSuiteFlag = true - TestingT(t) -} - func kindToFieldType(kind byte) types.FieldType { ft := types.FieldType{} switch kind { @@ -552,7 +545,7 @@ func TestUnaryOp(t *testing.T) { require.NoError(t, err) result, err := evalBuiltinFunc(f, chunk.Row{}) require.NoError(t, err) - require.Equal(t, types.NewDatum(tt.result), result, Commentf("%d", i)) + require.Equalf(t, types.NewDatum(tt.result), result, "%d", i) } tbl = []struct { @@ -576,7 +569,7 @@ func TestUnaryOp(t *testing.T) { expect := types.NewDatum(tt.result) ret, err := result.Compare(ctx.GetSessionVars().StmtCtx, &expect, collate.GetBinaryCollator()) require.NoError(t, err) - require.Equal(t, 0, ret, Commentf("%v %s", tt.arg, tt.op)) + require.Equalf(t, 0, ret, "%v %s", tt.arg, tt.op) } } diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index e1829b53338a3..1935cc6fbe27b 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" @@ -176,29 +175,29 @@ func TestColumn2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, colExprs, client) require.NoError(t, err) jsons := []string{ - "{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0,\"field_type\":{\"tp\":4,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0,\"field_type\":{\"tp\":6,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0,\"field_type\":{\"tp\":7,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0,\"field_type\":{\"tp\":9,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0,\"field_type\":{\"tp\":10,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0,\"field_type\":{\"tp\":11,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0,\"field_type\":{\"tp\":13,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0,\"field_type\":{\"tp\":245,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0,\"field_type\":{\"tp\":246,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0,\"field_type\":{\"tp\":249,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0,\"field_type\":{\"tp\":250,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0,\"field_type\":{\"tp\":251,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0,\"field_type\":{\"tp\":252,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", - "{\"tp\":201,\"val\":\"gAAAAAAAABg=\",\"sig\":0,\"field_type\":{\"tp\":247,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAQ=\",\"sig\":0,\"field_type\":{\"tp\":4,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAU=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAY=\",\"sig\":0,\"field_type\":{\"tp\":6,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAc=\",\"sig\":0,\"field_type\":{\"tp\":7,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAg=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAk=\",\"sig\":0,\"field_type\":{\"tp\":9,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAo=\",\"sig\":0,\"field_type\":{\"tp\":10,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAs=\",\"sig\":0,\"field_type\":{\"tp\":11,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAAw=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAA0=\",\"sig\":0,\"field_type\":{\"tp\":13,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAAA8=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABA=\",\"sig\":0,\"field_type\":{\"tp\":245,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABE=\",\"sig\":0,\"field_type\":{\"tp\":246,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABI=\",\"sig\":0,\"field_type\":{\"tp\":249,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABM=\",\"sig\":0,\"field_type\":{\"tp\":250,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABQ=\",\"sig\":0,\"field_type\":{\"tp\":251,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABU=\",\"sig\":0,\"field_type\":{\"tp\":252,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABY=\",\"sig\":0,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABc=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", + "{\"tp\":201,\"val\":\"gAAAAAAAABg=\",\"sig\":0,\"field_type\":{\"tp\":247,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", } for i, pbExpr := range pbExprs { require.NotNil(t, pbExprs) @@ -237,13 +236,13 @@ func TestCompareFunc2Pb(t *testing.T) { require.NoError(t, err) require.Len(t, pbExprs, len(compareExprs)) jsons := []string{ - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":110,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":120,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":130,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":140,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":150,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":160,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":110,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":120,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":130,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":140,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":150,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":160,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", } for i, pbExpr := range pbExprs { require.NotNil(t, pbExprs) @@ -281,8 +280,8 @@ func TestLikeFunc2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, likeFuncs, client) require.NoError(t, err) results := []string{ - `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false},{"tp":5,"val":"cGF0dGVybg==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":4310,"field_type":{"tp":8,"flag":524416,"flen":1,"decimal":0,"collate":63,"charset":"binary"},"has_distinct":false}`, - `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false},{"tp":5,"val":"JWFiYyU=","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"},"has_distinct":false}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"},"has_distinct":false}],"sig":4310,"field_type":{"tp":8,"flag":524416,"flen":1,"decimal":0,"collate":63,"charset":"binary"},"has_distinct":false}`, + `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false},{"tp":5,"val":"cGF0dGVybg==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":-83,"charset":"binary"},"has_distinct":false}],"sig":4310,"field_type":{"tp":8,"flag":524416,"flen":1,"decimal":0,"collate":-83,"charset":"binary"},"has_distinct":false}`, + `{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false},{"tp":5,"val":"JWFiYyU=","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":-83,"charset":"utf8"},"has_distinct":false}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":-83,"charset":"binary"},"has_distinct":false}],"sig":4310,"field_type":{"tp":8,"flag":524416,"flen":1,"decimal":0,"collate":-83,"charset":"binary"},"has_distinct":false}`, } for i, pbExpr := range pbExprs { js, err := json.Marshal(pbExpr) @@ -309,11 +308,11 @@ func TestArithmeticalFunc2Pb(t *testing.T) { } jsons := make(map[string]string) - jsons[ast.Plus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":200,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}" - jsons[ast.Minus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":204,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}" - jsons[ast.Mul] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":208,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}" - jsons[ast.Div] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":211,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}" - jsons[ast.Mod] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":215,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}" + jsons[ast.Plus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":200,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}" + jsons[ast.Minus] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":204,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}" + jsons[ast.Mul] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":208,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}" + jsons[ast.Div] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":211,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}" + jsons[ast.Mod] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":215,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}" pbExprs, err := ExpressionsToPBList(sc, arithmeticalFuncs, client) require.NoError(t, err) @@ -355,7 +354,7 @@ func TestDateFunc2Pb(t *testing.T) { require.NotNil(t, pbExprs[0]) js, err := json.Marshal(pbExprs[0]) require.NoError(t, err) - require.Equal(t, "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}],\"sig\":6001,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":0,\"decimal\":-1,\"collate\":46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", string(js)) + require.Equal(t, "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":12,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":254,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}],\"sig\":6001,\"field_type\":{\"tp\":253,\"flag\":0,\"flen\":0,\"decimal\":-1,\"collate\":-46,\"charset\":\"utf8mb4\"},\"has_distinct\":false}", string(js)) } func TestLogicalFunc2Pb(t *testing.T) { @@ -382,10 +381,10 @@ func TestLogicalFunc2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, logicalFuncs, client) require.NoError(t, err) jsons := []string{ - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3101,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3102,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3103,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3104,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3101,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3102,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3103,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3104,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", } for i, pbExpr := range pbExprs { js, err := json.Marshal(pbExpr) @@ -418,12 +417,12 @@ func TestBitwiseFunc2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, bitwiseFuncs, client) require.NoError(t, err) jsons := []string{ - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3118,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3119,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3120,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3129,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3130,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3121,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3118,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3119,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3120,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3129,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3130,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3121,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", } for i, pbExpr := range pbExprs { js, err := json.Marshal(pbExpr) @@ -461,9 +460,9 @@ func TestControlFunc2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, controlFuncs, client) require.NoError(t, err) jsons := []string{ - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4101,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4101,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":24,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", "null", } for i, pbExpr := range pbExprs { @@ -493,8 +492,8 @@ func TestOtherFunc2Pb(t *testing.T) { pbExprs, err := ExpressionsToPBList(sc, otherFuncs, client) require.NoError(t, err) jsons := map[string]string{ - ast.Coalesce: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4201,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", - ast.IsNull: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3116,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false}", + ast.Coalesce: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":4201,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", + ast.IsNull: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}],\"sig\":3116,\"field_type\":{\"tp\":8,\"flag\":524416,\"flen\":1,\"decimal\":0,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false}", } for i, pbExpr := range pbExprs { js, err := json.Marshal(pbExpr) @@ -1091,6 +1090,7 @@ func TestExprPushDownToTiKV(t *testing.T) { stringColumn := genColumn(mysql.TypeString, 5) //datetimeColumn := genColumn(mysql.TypeDatetime, 6) binaryStringColumn := genColumn(mysql.TypeString, 7) + dateColumn := genColumn(mysql.TypeDate, 8) binaryStringColumn.RetType.Collate = charset.CollationBin // Test exprs that cannot be pushed. @@ -1142,7 +1142,225 @@ func TestExprPushDownToTiKV(t *testing.T) { exprs = append(exprs, function) } - function, err = NewFunction(mock.NewContext(), ast.CharLength, types.NewFieldType(mysql.TypeString), stringColumn) + testcases := []struct { + functionName string + retType *types.FieldType + args []Expression + }{ + { + functionName: ast.CharLength, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.Right, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, intColumn}, + }, + { + functionName: ast.Left, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, intColumn}, + }, + { + functionName: ast.Sin, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Asin, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Cos, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Acos, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Tan, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Atan, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Cot, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Atan2, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn, intColumn}, + }, + { + functionName: ast.DateFormat, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn, stringColumn}, + }, + { + functionName: ast.Hour, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.Minute, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.Second, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.Month, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.MicroSecond, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.PI, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{}, + }, + { + functionName: ast.Round, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn}, + }, + { + functionName: ast.Truncate, + retType: types.NewFieldType(mysql.TypeDouble), + args: []Expression{intColumn, intColumn}, + }, + { + functionName: ast.Bin, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{intColumn}, + }, + { + functionName: ast.Unhex, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.Locate, + retType: types.NewFieldType(mysql.TypeInt24), + args: []Expression{stringColumn, stringColumn}, + }, + { + functionName: ast.Ord, + retType: types.NewFieldType(mysql.TypeInt24), + args: []Expression{stringColumn}, + }, + { + functionName: ast.Lpad, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, intColumn, stringColumn}, + }, + { + functionName: ast.Rpad, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, intColumn, stringColumn}, + }, + { + functionName: ast.Trim, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.FromBase64, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.ToBase64, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.MakeSet, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{intColumn, stringColumn}, + }, + { + functionName: ast.SubstringIndex, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, stringColumn, intColumn}, + }, + { + functionName: ast.Instr, + retType: types.NewFieldType(mysql.TypeInt24), + args: []Expression{stringColumn, stringColumn}, + }, + { + functionName: ast.Quote, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn}, + }, + { + functionName: ast.Oct, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{intColumn}, + }, + { + functionName: ast.FindInSet, + retType: types.NewFieldType(mysql.TypeInt24), + args: []Expression{stringColumn, stringColumn}, + }, + { + functionName: ast.Repeat, + retType: types.NewFieldType(mysql.TypeString), + args: []Expression{stringColumn, intColumn}, + }, + { + functionName: ast.Date, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.Week, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.YearWeek, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.ToSeconds, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn}, + }, + { + functionName: ast.DateDiff, + retType: types.NewFieldType(mysql.TypeDate), + args: []Expression{dateColumn, dateColumn}, + }, + } + + for _, tc := range testcases { + function, err = NewFunction(mock.NewContext(), tc.functionName, tc.retType, tc.args...) + require.NoError(t, err) + exprs = append(exprs, function) + } + + function, err = NewFunction(mock.NewContext(), ast.Mod, types.NewFieldType(mysql.TypeInt24), intColumn, intColumn) require.NoError(t, err) exprs = append(exprs, function) @@ -1185,13 +1403,13 @@ func TestGroupByItem2Pb(t *testing.T) { pbByItem := GroupByItemToPB(sc, client, item) js, err := json.Marshal(pbByItem) require.NoError(t, err) - require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) + require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) item = genColumn(mysql.TypeDouble, 1) pbByItem = GroupByItemToPB(sc, client, item) js, err = json.Marshal(pbByItem) require.NoError(t, err) - require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) + require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) } func TestSortByItem2Pb(t *testing.T) { @@ -1202,25 +1420,22 @@ func TestSortByItem2Pb(t *testing.T) { pbByItem := SortByItemToPB(sc, client, item, false) js, err := json.Marshal(pbByItem) require.NoError(t, err) - require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) + require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAA=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) item = genColumn(mysql.TypeDouble, 1) pbByItem = SortByItemToPB(sc, client, item, false) js, err = json.Marshal(pbByItem) require.NoError(t, err) - require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) + require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":false}", string(js)) item = genColumn(mysql.TypeDouble, 1) pbByItem = SortByItemToPB(sc, client, item, true) js, err = json.Marshal(pbByItem) require.NoError(t, err) - require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":true}", string(js)) + require.Equal(t, "{\"expr\":{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-63,\"charset\":\"binary\"},\"has_distinct\":false},\"desc\":true}", string(js)) } func TestPushCollationDown(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - fc, err := NewFunction(mock.NewContext(), ast.EQ, types.NewFieldType(mysql.TypeUnspecified), genColumn(mysql.TypeVarchar, 0), genColumn(mysql.TypeVarchar, 1)) require.NoError(t, err) client := new(mock.Client) @@ -1245,8 +1460,6 @@ func columnCollation(c *Column, chs, coll string) *Column { } func TestNewCollationsEnabled(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) var colExprs []Expression sc := new(stmtctx.StatementContext) client := new(mock.Client) @@ -1346,7 +1559,7 @@ func TestPushDownSwitcher(t *testing.T) { } var enabled []string for _, funcName := range cases { - args := []Expression{genColumn(mysql.TypeLong, 1)} + args := []Expression{genColumn(mysql.TypeDouble, 1)} fc, err := NewFunction( mock.NewContext(), funcName.name, diff --git a/expression/expression.go b/expression/expression.go index 14e29c6a795d7..f8155785757dc 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -941,6 +941,7 @@ func IsBinaryLiteral(expr Expression) bool { return ok && con.Value.Kind() == types.KindBinaryLiteral } +// supported functions tracked by https://github.com/tikv/tikv/issues/5751 func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { switch sf.FuncName.L { case @@ -951,23 +952,29 @@ func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { ast.LT, ast.LE, ast.EQ, ast.NE, ast.GE, ast.GT, ast.NullEQ, ast.In, ast.IsNull, ast.Like, ast.IsTruthWithoutNull, ast.IsTruthWithNull, ast.IsFalsity, // arithmetical functions. - ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Abs, /*ast.Mod,*/ + ast.PI, ast.Truncate, + ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Abs, ast.Mod, // math functions. ast.Ceil, ast.Ceiling, ast.Floor, ast.Sqrt, ast.Sign, ast.Ln, ast.Log, ast.Log2, ast.Log10, ast.Exp, ast.Pow, // Rust use the llvm math functions, which have different precision with Golang/MySQL(cmath) // open the following switchers if we implement them in coprocessor via `cmath` - // ast.Sin, ast.Asin, ast.Cos, ast.Acos, ast.Tan, ast.Atan, ast.Atan2, ast.Cot, + ast.Sin, ast.Asin, ast.Cos, ast.Acos, ast.Tan, ast.Atan, ast.Atan2, ast.Cot, ast.Radians, ast.Degrees, ast.Conv, ast.CRC32, // control flow functions. ast.Case, ast.If, ast.Ifnull, ast.Coalesce, // string functions. - ast.Length, ast.BitLength, ast.Concat, ast.ConcatWS /*ast.Locate,*/, ast.Replace, ast.ASCII, ast.Hex, - ast.Reverse, ast.LTrim, ast.RTrim /*ast.Left,*/, ast.Strcmp, ast.Space, ast.Elt, ast.Field, + ast.Bin, ast.Unhex, ast.Locate, ast.Ord, ast.Lpad, ast.Rpad, + ast.Trim, ast.FromBase64, ast.ToBase64, /* ast.Upper, ast.Lower, ast.InsertFunc */ + ast.MakeSet, ast.SubstringIndex, ast.Instr, ast.Quote, ast.Oct, + ast.FindInSet, ast.Repeat, + ast.Length, ast.BitLength, ast.Concat, ast.ConcatWS, ast.Replace, ast.ASCII, ast.Hex, + ast.Reverse, ast.LTrim, ast.RTrim, ast.Strcmp, ast.Space, ast.Elt, ast.Field, InternalFuncFromBinary, InternalFuncToBinary, ast.Mid, ast.Substring, ast.Substr, ast.CharLength, + ast.Right, ast.Left, // json functions. ast.JSONType, ast.JSONExtract, ast.JSONObject, ast.JSONArray, ast.JSONMerge, ast.JSONSet, @@ -976,10 +983,17 @@ func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { ast.JSONUnquote, // date functions. - ast.DateFormat, ast.FromDays /*ast.ToDays,*/, ast.DayOfYear, ast.DayOfMonth, ast.Year, ast.Month, - // FIXME: the coprocessor cannot keep the same behavior with TiDB in current compute framework - // ast.Hour, ast.Minute, ast.Second, ast.MicroSecond, ast.DayName, + ast.Date, ast.Week, ast.YearWeek, ast.ToSeconds, ast.DateDiff, + /* ast.TimeDiff, ast.AddTime, ast.SubTime, */ + ast.MonthName, ast.MakeDate, ast.TimeToSec, ast.MakeTime, + ast.DateFormat, + ast.Hour, ast.Minute, ast.Second, ast.MicroSecond, ast.Month, + /* ast.DayName */ ast.DayOfMonth, ast.DayOfWeek, ast.DayOfYear, + ast.Weekday, ast.WeekOfYear, ast.Year, + ast.FromDays, ast.ToDays, ast.PeriodAdd, ast.PeriodDiff, /*ast.TimestampDiff, ast.DateAdd, ast.FromUnixTime,*/ + ast.LastDay, + ast.Sysdate, // encryption functions. ast.MD5, ast.SHA1, ast.UncompressedLength, @@ -992,11 +1006,10 @@ func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { ast.UUID: return true - - // A special case: Only push down Round by signature case ast.Round: switch sf.Function.PbCode() { - case tipb.ScalarFuncSig_RoundReal, tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundDec: + case tipb.ScalarFuncSig_RoundReal, tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundDec, + tipb.ScalarFuncSig_RoundWithFracReal, tipb.ScalarFuncSig_RoundWithFracInt, tipb.ScalarFuncSig_RoundWithFracDec: return true } case ast.Rand: @@ -1162,6 +1175,7 @@ func canFuncBePushed(sf *ScalarFunction, storeType kv.StoreType) bool { failpoint.Return(true) } } + failpoint.Return(false) }) ret := false diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go index fcd53107db523..86a8f6703d8d8 100644 --- a/expression/generator/time_vec.go +++ b/expression/generator/time_vec.go @@ -242,7 +242,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result {{ end }} {{ else if or (eq .SigName "builtinAddStringAndDurationSig") (eq .SigName "builtinSubStringAndDurationSig") }} sc := b.ctx.GetSessionVars().StmtCtx - fsp1 := int8(b.args[1].GetType().Decimal) + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} var output string var isNull bool @@ -309,8 +309,8 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result } } {{ else if or (eq .SigName "builtinAddDateAndDurationSig") (eq .SigName "builtinSubDateAndDurationSig") }} - fsp0 := int8(b.args[0].GetType().Decimal) - fsp1 := int8(b.args[1].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal + fsp1 := b.args[1].GetType().Decimal arg1Duration := types.Duration{Duration: arg1, Fsp: fsp1} {{ if eq $.FuncName "AddTime" }} sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) @@ -323,7 +323,7 @@ func (b *{{.SigName}}) vecEval{{ .Output.TypeName }}(input *chunk.Chunk, result output := sum.String() {{ else if or (eq .SigName "builtinAddDateAndStringSig") (eq .SigName "builtinSubDateAndStringSig") }} {{ template "ConvertStringToDuration" . }} - fsp0 := int8(b.args[0].GetType().Decimal) + fsp0 := b.args[0].GetType().Decimal {{ if eq $.FuncName "AddTime" }} sum, err := types.Duration{Duration: arg0, Fsp: fsp0}.Add(arg1Duration) {{ else }} @@ -436,7 +436,7 @@ func (b *{{.SigName}}) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) {{ if $BIsDuration }} lhsDur, _, lhsIsDuration, {{- else if $BIsTime }} _, lhsTime, lhsIsDuration, {{- else if $BIsString }} lhsDur, lhsTime, lhsIsDuration, - {{- end }} err := convertStringToDuration(stmtCtx, buf0.GetString(i), int8(b.tp.Decimal)) + {{- end }} err := convertStringToDuration(stmtCtx, buf0.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -462,7 +462,7 @@ func (b *{{.SigName}}) vecEvalDuration(input *chunk.Chunk, result *chunk.Column) {{ if $AIsDuration }} rhsDur, _, rhsIsDuration, {{- else if $AIsTime }}_, rhsTime, rhsIsDuration, {{- else if $AIsString }} rhsDur, rhsTime, rhsIsDuration, - {{- end}} err := convertStringToDuration(stmtCtx, buf1.GetString(i), int8(b.tp.Decimal)) + {{- end}} err := convertStringToDuration(stmtCtx, buf1.GetString(i), b.tp.Decimal) if err != nil { return err } @@ -739,7 +739,7 @@ func (g gener) gen() interface{} { if _, ok := result.(string); ok { dg := newDefaultGener(0, types.ETDuration) d := dg.gen().(types.Duration) - if int8(d.Duration)%2 == 0 { + if d.Duration%2 == 0 { d.Fsp = 0 } else { d.Fsp = 1 diff --git a/expression/helper.go b/expression/helper.go index 6d1710ccbdba5..c292b954f1723 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -57,7 +57,7 @@ func IsValidCurrentTimestampExpr(exprNode ast.ExprNode, fieldType *types.FieldTy } // GetTimeCurrentTimestamp is used for generating a timestamp for some special cases: cast null value to timestamp type with not null flag. -func GetTimeCurrentTimestamp(ctx sessionctx.Context, tp byte, fsp int8) (d types.Datum, err error) { +func GetTimeCurrentTimestamp(ctx sessionctx.Context, tp byte, fsp int) (d types.Datum, err error) { var t types.Time t, err = getTimeCurrentTimeStamp(ctx, tp, fsp) if err != nil { @@ -67,13 +67,13 @@ func GetTimeCurrentTimestamp(ctx sessionctx.Context, tp byte, fsp int8) (d types return d, nil } -func getTimeCurrentTimeStamp(ctx sessionctx.Context, tp byte, fsp int8) (t types.Time, err error) { +func getTimeCurrentTimeStamp(ctx sessionctx.Context, tp byte, fsp int) (t types.Time, err error) { value := types.NewTime(types.ZeroCoreTime, tp, fsp) defaultTime, err := getStmtTimestamp(ctx) if err != nil { return value, err } - value.SetCoreTime(types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-int(fsp))) * time.Nanosecond))) + value.SetCoreTime(types.FromGoTime(defaultTime.Truncate(time.Duration(math.Pow10(9-fsp)) * time.Nanosecond))) if tp == mysql.TypeTimestamp || tp == mysql.TypeDatetime { err = value.ConvertTimeZone(time.Local, ctx.GetSessionVars().Location()) if err != nil { @@ -84,7 +84,7 @@ func getTimeCurrentTimeStamp(ctx sessionctx.Context, tp byte, fsp int8) (t types } // GetTimeValue gets the time value with type tp. -func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int8) (d types.Datum, err error) { +func GetTimeValue(ctx sessionctx.Context, v interface{}, tp byte, fsp int) (d types.Datum, err error) { var value types.Time sc := ctx.GetSessionVars().StmtCtx diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index 01013e56a0283..9b6751e2f5893 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" @@ -82,9 +81,6 @@ func TestIssue17727(t *testing.T) { } func TestIssue17891(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -97,9 +93,6 @@ func TestIssue17891(t *testing.T) { } func TestIssue31174(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -120,9 +113,6 @@ func TestIssue31174(t *testing.T) { } func TestIssue20268(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -139,8 +129,6 @@ func TestCollationBasic(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("use test") tk.MustExec("create table t_ci(a varchar(10) collate utf8mb4_general_ci, unique key(a))") tk.MustExec("insert into t_ci values ('a')") @@ -216,8 +204,6 @@ func TestWeightString(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) type testCase struct { input []string @@ -291,8 +277,6 @@ func TestCollationCreateIndex(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(10) collate utf8mb4_general_ci);") @@ -326,8 +310,6 @@ func TestCollateConstantPropagation(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -398,8 +380,6 @@ func TestMixCollation(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin = 'a' collate utf8mb4_general_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_general_ci,EXPLICIT) for operation '='") @@ -491,8 +471,6 @@ func prepare4Join(tk *testkit.TestKit) { } func TestCollateHashJoin(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -517,8 +495,6 @@ func TestCollateHashJoin(t *testing.T) { } func TestCollateHashJoin2(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -539,8 +515,6 @@ func prepare4Join2(tk *testkit.TestKit) { } func TestCollateMergeJoin(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -565,8 +539,6 @@ func TestCollateMergeJoin(t *testing.T) { } func TestCollateMergeJoin2(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -577,8 +549,6 @@ func TestCollateMergeJoin2(t *testing.T) { } func TestCollateIndexMergeJoin(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -594,8 +564,6 @@ func TestCollateIndexMergeJoin(t *testing.T) { } func TestNewCollationCheckClusterIndexTable(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -623,8 +591,6 @@ func prepare4Collation(tk *testkit.TestKit, hasIndex bool) { } func TestCollateSelection(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -637,8 +603,6 @@ func TestCollateSelection(t *testing.T) { } func TestCollateSort(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -658,8 +622,6 @@ func TestCollateSort(t *testing.T) { } func TestCollateHashAgg(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -686,8 +648,6 @@ func TestCollateHashAgg(t *testing.T) { } func TestCollateStreamAgg(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -704,8 +664,6 @@ func TestCollateStreamAgg(t *testing.T) { } func TestCollateIndexReader(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -722,8 +680,6 @@ func TestCollateIndexReader(t *testing.T) { } func TestCollateIndexLookup(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -741,8 +697,6 @@ func TestCollateIndexLookup(t *testing.T) { } func TestIssue16668(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -755,8 +709,6 @@ func TestIssue16668(t *testing.T) { } func TestIssue27091(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -774,8 +726,6 @@ func TestIssue27091(t *testing.T) { } func TestCollateStringFunction(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -901,9 +851,6 @@ func TestCollateStringFunction(t *testing.T) { } func TestCollateLike(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -937,8 +884,6 @@ func TestCollateLike(t *testing.T) { } func TestCollateSubQuery(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -955,8 +900,6 @@ func TestCollateSubQuery(t *testing.T) { } func TestCollateDDL(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -967,8 +910,6 @@ func TestCollateDDL(t *testing.T) { } func TestNewCollationWithClusterIndex(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -982,8 +923,6 @@ func TestNewCollationWithClusterIndex(t *testing.T) { } func TestNewCollationBinaryFlag(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1027,8 +966,6 @@ func TestNewCollationBinaryFlag(t *testing.T) { } func TestIssue17176(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1044,9 +981,6 @@ func TestIssue17176(t *testing.T) { } func TestIssue18638(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1060,9 +994,6 @@ func TestIssue18638(t *testing.T) { } func TestCollationText(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1077,9 +1008,6 @@ func TestCollationText(t *testing.T) { } func TestIssue18662(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1095,9 +1023,6 @@ func TestIssue18662(t *testing.T) { } func TestIssue19116(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1122,9 +1047,6 @@ func TestIssue19116(t *testing.T) { } func TestIssue17063(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1142,9 +1064,6 @@ func TestIssue17063(t *testing.T) { } func TestIssue11177(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1160,9 +1079,6 @@ func TestIssue11177(t *testing.T) { } func TestIssue19804(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1182,9 +1098,6 @@ func TestIssue19804(t *testing.T) { } func TestIssue20209(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1196,9 +1109,6 @@ func TestIssue20209(t *testing.T) { } func TestIssue18949(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1214,9 +1124,6 @@ func TestIssue18949(t *testing.T) { } func TestClusteredIndexAndNewCollationIndexEncodeDecodeV5(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1287,9 +1194,6 @@ func TestClusteredIndexAndNewCollationIndexEncodeDecodeV5(t *testing.T) { } func TestClusteredIndexAndNewCollation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1321,8 +1225,6 @@ func TestClusteredIndexAndNewCollation(t *testing.T) { } func TestIssue20608(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1331,9 +1233,6 @@ func TestIssue20608(t *testing.T) { } func TestIssue20161(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -1347,8 +1246,6 @@ func TestIssue20161(t *testing.T) { } func TestCollationIndexJoin(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1384,8 +1281,6 @@ func TestCollationIndexJoin(t *testing.T) { } func TestCollationMergeJoin(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1413,8 +1308,6 @@ func TestCollationMergeJoin(t *testing.T) { } func TestIssue20876(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1439,9 +1332,6 @@ func TestLikeWithCollation(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustQuery(`select 'a' like 'A' collate utf8mb4_unicode_ci;`).Check(testkit.Rows("1")) tk.MustGetErrMsg(`select 'a' collate utf8mb4_bin like 'A' collate utf8mb4_unicode_ci;`, "[expression:1267]Illegal mix of collations (utf8mb4_bin,EXPLICIT) and (utf8mb4_unicode_ci,EXPLICIT) for operation 'like'") tk.MustQuery(`select '😛' collate utf8mb4_general_ci like '😋';`).Check(testkit.Rows("1")) @@ -1450,31 +1340,12 @@ func TestLikeWithCollation(t *testing.T) { tk.MustQuery(`select '😛' collate utf8mb4_unicode_ci = '😋';`).Check(testkit.Rows("1")) } -func TestCollationUnion(t *testing.T) { - // For issue 19694. - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - - tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ' order by a;").Check(testkit.Rows("2010-09-09", "2010-09-09 ")) - res := tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") - require.Len(t, res.Rows(), 2) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - res = tk.MustQuery("select cast('2010-09-09' as date) a union select '2010-09-09 ';") - require.Len(t, res.Rows(), 1) -} - func TestCollationPrefixClusteredIndex(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustExec("drop table if exists t") tk.MustExec("create table t (k char(20), v int, primary key (k(4)) clustered, key (k)) collate utf8mb4_general_ci;") tk.MustExec("insert into t values('01233', 1);") @@ -1490,9 +1361,6 @@ func TestIssue23805(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk.MustExec("CREATE TABLE `tbl_5` (" + " `col_25` time NOT NULL DEFAULT '05:35:58'," + " `col_26` blob NOT NULL," + @@ -1508,8 +1376,6 @@ func TestIssue23805(t *testing.T) { } func TestIssue26662(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1523,8 +1389,6 @@ func TestIssue26662(t *testing.T) { } func TestIssue30245(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) store, clean := testkit.CreateMockStore(t) defer clean() @@ -1539,8 +1403,6 @@ func TestCollationForBinaryLiteral(t *testing.T) { defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE t (`COL1` tinyblob NOT NULL, `COL2` binary(1) NOT NULL, `COL3` bigint(11) NOT NULL, PRIMARY KEY (`COL1`(5),`COL2`,`COL3`) /*T![clustered_index] CLUSTERED */)") @@ -2050,7 +1912,7 @@ func TestTimeBuiltin(t *testing.T) { _, err = tk.Exec(`update t set a = week("aa", 1)`) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue)) _, err = tk.Exec(`delete from t where a = week("aa", 1)`) - require.True(t, terror.ErrorEqual(err, types.ErrWrongValue)) + require.Equal(t, types.ErrWrongValue.Code(), errors.Cause(err).(*terror.Error).Code(), "err %v", err) // for weekofyear result = tk.MustQuery(`select weekofyear("2012-12-22"), weekofyear("2008-02-20"), weekofyear("aa"), weekofyear(null), weekofyear(11), weekofyear(12.99);`) @@ -2065,7 +1927,7 @@ func TestTimeBuiltin(t *testing.T) { _, err = tk.Exec(`update t set a = weekofyear("aa")`) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue)) _, err = tk.Exec(`delete from t where a = weekofyear("aa")`) - require.True(t, terror.ErrorEqual(err, types.ErrWrongValue)) + require.Equal(t, types.ErrWrongValue.Code(), errors.Cause(err).(*terror.Error).Code(), "err %v", err) // for weekday result = tk.MustQuery(`select weekday("2012-12-20"), weekday("2012-12-21"), weekday("2012-12-22"), weekday("2012-12-23"), weekday("2012-12-24"), weekday("2012-12-25"), weekday("2012-12-26"), weekday("2012-12-27");`) @@ -3913,7 +3775,7 @@ func TestPreparePlanCache(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) } -func TestPreparePlanCacheNotForCacheTable(t *testing.T) { +func TestPreparePlanCacheOnCachedTable(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -3934,28 +3796,24 @@ func TestPreparePlanCacheNotForCacheTable(t *testing.T) { tk.MustExec("create table t(a int);") tk.MustExec("alter table t cache") - var useCache bool + var readFromTableCache bool for i := 0; i < 50; i++ { tk.MustQuery("select * from t where a = 1") - if tk.HasPlan("select * from t where a = 1", "Union") { - useCache = true + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + readFromTableCache = true + break } } - require.True(t, useCache) + require.True(t, readFromTableCache) // already read cache after reading first time - tk.MustQuery("explain format = 'brief' select * from t where a = 1").Check(testkit.Rows( - "Projection 10.00 root test.t.a", - "└─UnionScan 10.00 root eq(test.t.a, 1)", - " └─TableReader 10.00 root data:Selection", - " └─Selection 10.00 cop[tikv] eq(test.t.a, 1)", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) - tk.MustExec("prepare stmt from 'select * from t where a = ?';") tk.MustExec("set @a = 1;") tk.MustExec("execute stmt using @a;") tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) tk.MustExec("execute stmt using @a;") - tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + readFromTableCache = tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache + require.True(t, readFromTableCache) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) } func TestIssue16205(t *testing.T) { @@ -4568,8 +4426,8 @@ PARTITION BY RANGE (c) ( tk.MustExec("set global tidb_enable_local_txn = on;") for _, testcase := range testcases { t.Log(testcase.name) - failpoint.Enable("tikvclient/injectTxnScope", - fmt.Sprintf(`return("%v")`, testcase.zone)) + require.NoError(t, failpoint.Enable("tikvclient/injectTxnScope", + fmt.Sprintf(`return("%v")`, testcase.zone))) tk.MustExec(fmt.Sprintf("set @@txn_scope='%v'", testcase.txnScope)) tk.Exec("begin") res, err := tk.Exec(testcase.sql) @@ -4591,7 +4449,7 @@ PARTITION BY RANGE (c) ( } tk.Exec("commit") } - failpoint.Disable("tikvclient/injectTxnScope") + require.NoError(t, failpoint.Disable("tikvclient/injectTxnScope")) tk.MustExec("set global tidb_enable_local_txn = off;") } diff --git a/expression/integration_test.go b/expression/integration_test.go index e0131d6265ac6..7091f50818c4f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/sqlexec" @@ -2776,6 +2777,9 @@ func TestTiDBDecodeKeyFunc(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() + collate.SetNewCollationEnabledForTest(false) + defer collate.SetNewCollationEnabledForTest(true) + tk := testkit.NewTestKit(t, store) var result *testkit.Result @@ -3479,36 +3483,33 @@ func TestExprPushdown(t *testing.T) { // case 1, index scan without double read, some filters can not be pushed to cop task rows := tk.MustQuery("explain format = 'brief' select col2, col1 from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Rows() - require.Equal(t, "root", fmt.Sprintf("%v", rows[1][2])) - require.Equal(t, "eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), \"4\")", fmt.Sprintf("%v", rows[1][4])) - require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[3][2])) - require.Equal(t, "like(test.t.col2, \"5%\", 92)", fmt.Sprintf("%v", rows[3][4])) + require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[2][2])) + require.Equal(t, "eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), \"4\"), like(test.t.col2, \"5%\", 92)", fmt.Sprintf("%v", rows[2][4])) tk.MustQuery("select col2, col1 from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("511 411111")) tk.MustQuery("select count(col2) from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("1")) // case 2, index scan without double read, none of the filters can be pushed to cop task rows = tk.MustQuery("explain format = 'brief' select col1, col2 from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Rows() - require.Equal(t, "root", fmt.Sprintf("%v", rows[0][2])) - require.Equal(t, "eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), \"4\"), eq(from_base64(to_base64(substr(test.t.col2, 1, 1))), \"5\")", fmt.Sprintf("%v", rows[0][4])) + require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[1][2])) + require.Equal(t, `eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), "4"), eq(from_base64(to_base64(substr(test.t.col2, 1, 1))), "5")`, fmt.Sprintf("%v", rows[1][4])) tk.MustQuery("select col1, col2 from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("411111 511")) tk.MustQuery("select count(col1) from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("1")) // case 3, index scan with double read, some filters can not be pushed to cop task rows = tk.MustQuery("explain format = 'brief' select id from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Rows() - require.Equal(t, "root", fmt.Sprintf("%v", rows[1][2])) - require.Equal(t, "eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), \"4\")", fmt.Sprintf("%v", rows[1][4])) - require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[3][2])) - require.Equal(t, "like(test.t.col2, \"5%\", 92)", fmt.Sprintf("%v", rows[3][4])) + require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[2][2])) + require.Equal(t, `eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), "4"), like(test.t.col2, "5%", 92)`, fmt.Sprintf("%v", rows[2][4])) tk.MustQuery("select id from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("3")) tk.MustQuery("select count(id) from t use index(key1) where col2 like '5%' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("1")) // case 4, index scan with double read, none of the filters can be pushed to cop task rows = tk.MustQuery("explain format = 'brief' select id from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Rows() - require.Equal(t, "root", fmt.Sprintf("%v", rows[1][2])) - require.Equal(t, "eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), \"4\"), eq(from_base64(to_base64(substr(test.t.col2, 1, 1))), \"5\")", fmt.Sprintf("%v", rows[1][4])) + require.Equal(t, "cop[tikv]", fmt.Sprintf("%v", rows[2][2])) + require.Equal(t, `eq(from_base64(to_base64(substr(test.t.col1, 1, 1))), "4"), eq(from_base64(to_base64(substr(test.t.col2, 1, 1))), "5")`, fmt.Sprintf("%v", rows[2][4])) tk.MustQuery("select id from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("3")) tk.MustQuery("select count(id) from t use index(key2) where from_base64(to_base64(substr(col2, 1, 1))) = '5' and from_base64(to_base64(substr(col1, 1, 1))) = '4'").Check(testkit.Rows("1")) } + func TestIssue16973(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -4175,7 +4176,7 @@ func TestSelectLimitPlanCache(t *testing.T) { tk.MustQuery("execute stmt").Check(testkit.Rows("1", "2")) } -func TestCollation(t *testing.T) { +func TestCollationAndCharset(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -4185,51 +4186,51 @@ func TestCollation(t *testing.T) { tk.MustExec("create table t (utf8_bin_c varchar(10) charset utf8 collate utf8_bin, utf8_gen_c varchar(10) charset utf8 collate utf8_general_ci, bin_c binary, num_c int, " + "abin char collate ascii_bin, lbin char collate latin1_bin, u4bin char collate utf8mb4_bin, u4ci char collate utf8mb4_general_ci)") tk.MustExec("insert into t values ('a', 'b', 'c', 4, 'a', 'a', 'a', 'a')") - tk.MustQuery("select collation(null)").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(2)").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(2 + 'a')").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(2 + utf8_gen_c) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(2 + utf8_bin_c) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(utf8_bin_c, 2)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(concat(utf8_gen_c, 'abc')) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(concat(utf8_gen_c, null)) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(concat(utf8_gen_c, num_c)) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(concat(utf8_bin_c, utf8_gen_c)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(upper(utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(upper(utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(upper(bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(abin, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(lbin, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(utf8_bin_c, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(utf8_gen_c, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(u4bin, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(u4ci, bin_c)) from t").Check(testkit.Rows("binary")) - tk.MustQuery("select collation(concat(abin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select collation(concat(lbin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select collation(concat(utf8_bin_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select collation(concat(utf8_gen_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select collation(concat(u4ci, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin")) - tk.MustQuery("select collation(concat(abin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(concat(lbin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(concat(utf8_bin_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(concat(utf8_gen_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci")) - tk.MustQuery("select collation(concat(abin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(concat(lbin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(concat(utf8_gen_c, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin")) - tk.MustQuery("select collation(concat(abin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(concat(lbin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci")) - tk.MustQuery("select collation(concat(abin, lbin)) from t").Check(testkit.Rows("latin1_bin")) + tk.MustQuery("select collation(null), charset(null)").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(2), charset(2)").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(2 + 'a'), charset(2 + 'a')").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(2 + utf8_gen_c), charset(2 + utf8_gen_c) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(2 + utf8_bin_c), charset(2 + utf8_bin_c) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(utf8_bin_c, 2)), charset(concat(utf8_bin_c, 2)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(concat(utf8_gen_c, 'abc')), charset(concat(utf8_gen_c, 'abc')) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(concat(utf8_gen_c, null)), charset(concat(utf8_gen_c, null)) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(concat(utf8_gen_c, num_c)), charset(concat(utf8_gen_c, num_c)) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(concat(utf8_bin_c, utf8_gen_c)), charset(concat(utf8_bin_c, utf8_gen_c)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(upper(utf8_bin_c)), charset(upper(utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(upper(utf8_gen_c)), charset(upper(utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(upper(bin_c)), charset(upper(bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(abin, bin_c)), charset(concat(abin, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(lbin, bin_c)), charset(concat(lbin, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(utf8_bin_c, bin_c)), charset(concat(utf8_bin_c, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(utf8_gen_c, bin_c)), charset(concat(utf8_gen_c, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(u4bin, bin_c)), charset(concat(u4bin, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(u4ci, bin_c)), charset(concat(u4ci, bin_c)) from t").Check(testkit.Rows("binary binary")) + tk.MustQuery("select collation(concat(abin, u4bin)), charset(concat(abin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin utf8mb4")) + tk.MustQuery("select collation(concat(lbin, u4bin)), charset(concat(lbin, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin utf8mb4")) + tk.MustQuery("select collation(concat(utf8_bin_c, u4bin)), charset(concat(utf8_bin_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin utf8mb4")) + tk.MustQuery("select collation(concat(utf8_gen_c, u4bin)), charset(concat(utf8_gen_c, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin utf8mb4")) + tk.MustQuery("select collation(concat(u4ci, u4bin)), charset(concat(u4ci, u4bin)) from t").Check(testkit.Rows("utf8mb4_bin utf8mb4")) + tk.MustQuery("select collation(concat(abin, u4ci)), charset(concat(abin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) + tk.MustQuery("select collation(concat(lbin, u4ci)), charset(concat(lbin, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) + tk.MustQuery("select collation(concat(utf8_bin_c, u4ci)), charset(concat(utf8_bin_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) + tk.MustQuery("select collation(concat(utf8_gen_c, u4ci)), charset(concat(utf8_gen_c, u4ci)) from t").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) + tk.MustQuery("select collation(concat(abin, utf8_bin_c)), charset(concat(abin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(concat(lbin, utf8_bin_c)), charset(concat(lbin, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(concat(utf8_gen_c, utf8_bin_c)), charset(concat(utf8_gen_c, utf8_bin_c)) from t").Check(testkit.Rows("utf8_bin utf8")) + tk.MustQuery("select collation(concat(abin, utf8_gen_c)), charset(concat(abin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(concat(lbin, utf8_gen_c)), charset(concat(lbin, utf8_gen_c)) from t").Check(testkit.Rows("utf8_general_ci utf8")) + tk.MustQuery("select collation(concat(abin, lbin)), charset(concat(abin, lbin)) from t").Check(testkit.Rows("latin1_bin latin1")) tk.MustExec("set names utf8mb4 collate utf8mb4_bin") - tk.MustQuery("select collation('a')").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation('a'), charset('a')").Check(testkit.Rows("utf8mb4_bin utf8mb4")) tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") - tk.MustQuery("select collation('a')").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation('a'), charset('a')").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci") tk.MustExec("set @test_collate_var = 'a'") - tk.MustQuery("select collation(@test_collate_var)").Check(testkit.Rows("utf8mb4_general_ci")) + tk.MustQuery("select collation(@test_collate_var), charset(@test_collate_var)").Check(testkit.Rows("utf8mb4_general_ci utf8mb4")) tk.MustExec("set @test_collate_var = concat(\"a\", \"b\" collate utf8mb4_bin)") - tk.MustQuery("select collation(@test_collate_var)").Check(testkit.Rows("utf8mb4_bin")) + tk.MustQuery("select collation(@test_collate_var), charset(@test_collate_var)").Check(testkit.Rows("utf8mb4_bin utf8mb4")) tk.MustQuery("select locate('1', '123' collate utf8mb4_bin, 2 collate `binary`);").Check(testkit.Rows("0")) tk.MustQuery("select 1 in ('a' collate utf8mb4_bin, 'b' collate utf8mb4_general_ci);").Check(testkit.Rows("0")) @@ -4763,17 +4764,6 @@ func TestIssue17287(t *testing.T) { tk.MustQuery("execute stmt7 using @val2;").Check(testkit.Rows("1589873946")) } -func TestIssue26989(t *testing.T) { - store, clean := testkit.CreateMockStore(t) - defer clean() - - tk := testkit.NewTestKit(t, store) - tk.MustExec("set names utf8mb4 collate utf8mb4_general_ci;") - tk.MustQuery("select position('a' in 'AA');").Check(testkit.Rows("0")) - tk.MustQuery("select locate('a', 'AA');").Check(testkit.Rows("0")) - tk.MustQuery("select locate('a', 'a');").Check(testkit.Rows("1")) -} - func TestIssue17898(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -7027,7 +7017,7 @@ func TestIssue29708(t *testing.T) { }) _, err = tk.Exec("INSERT IGNORE INTO t1 VALUES (REPEAT(0125,200000000));") - require.Nil(t, err) + require.NoError(t, err) tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1301 Result of repeat() was larger than max_allowed_packet (67108864) - truncated")) tk.MustQuery("select a from t1 order by a;").Check([][]interface{}{ {nil}, @@ -7035,3 +7025,37 @@ func TestIssue29708(t *testing.T) { {"b"}, }) } + +func TestIssue22206(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tz := tk.Session().GetSessionVars().StmtCtx.TimeZone + result := tk.MustQuery("select from_unixtime(32536771199.999999)") + unixTime := time.Unix(32536771199, 999999000).In(tz).String()[:26] + result.Check(testkit.Rows(unixTime)) + result = tk.MustQuery("select from_unixtime('32536771200.000000')") + result.Check(testkit.Rows("")) + result = tk.MustQuery("select from_unixtime(5000000000);") + unixTime = time.Unix(5000000000, 0).In(tz).String()[:19] + result.Check(testkit.Rows(unixTime)) +} + +func TestIssue32488(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a varchar(32)) DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci;") + tk.MustExec("insert into t values('ʞ'), ('İ');") + tk.MustExec("set @@tidb_enable_vectorized_expression = false;") + tk.MustQuery("select binary upper(a), lower(a) from t order by upper(a);").Check([][]interface{}{{"İ i"}, {"Ʞ ʞ"}}) + tk.MustQuery("select distinct upper(a), lower(a) from t order by upper(a);").Check([][]interface{}{{"İ i"}, {"Ʞ ʞ"}}) + tk.MustExec("set @@tidb_enable_vectorized_expression = true;") + tk.MustQuery("select binary upper(a), lower(a) from t order by upper(a);").Check([][]interface{}{{"İ i"}, {"Ʞ ʞ"}}) + tk.MustQuery("select distinct upper(a), lower(a) from t order by upper(a);").Check([][]interface{}{{"İ i"}, {"Ʞ ʞ"}}) +} diff --git a/expression/main_test.go b/expression/main_test.go index 590c1c33ba72f..9fa0037ca462c 100644 --- a/expression/main_test.go +++ b/expression/main_test.go @@ -52,7 +52,7 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "expression_suite") opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/expression/scalar_function.go b/expression/scalar_function.go index bbc3086ed944f..5c2332c80a51f 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -187,6 +187,10 @@ func newFunctionImpl(ctx sessionctx.Context, fold int, funcName string, retType return BuildFromBinaryFunction(ctx, args[0], retType), nil case InternalFuncToBinary: return BuildToBinaryFunction(ctx, args[0]), nil + case ast.Sysdate: + if ctx.GetSessionVars().SysdateIsNow { + funcName = ast.Now + } } fc, ok := funcs[funcName] if !ok { diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 577ac1f7b2e15..89dac1cdfc23e 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -1223,7 +1223,7 @@ func (s *InferTypeSuite) createTestCase4OtherFuncs() []typeInferTestCase { {"bit_count(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, {"bit_count(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 2, 0}, - {`@varname`, mysql.TypeVarString, charset.CharsetUTF8MB4, 0, mysql.MaxFieldVarCharLength, int(types.UnspecifiedFsp)}, + {`@varname`, mysql.TypeVarString, charset.CharsetUTF8MB4, 0, mysql.MaxFieldVarCharLength, types.UnspecifiedFsp}, } } @@ -1803,9 +1803,9 @@ func (s *InferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"quarter(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, {"quarter(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, - {"current_time()", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 8, int(types.MinFsp)}, - {"current_time(0)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 8, int(types.MinFsp)}, - {"current_time(6)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 15, int(types.MaxFsp)}, + {"current_time()", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 8, types.MinFsp}, + {"current_time(0)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 8, types.MinFsp}, + {"current_time(6)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 15, types.MaxFsp}, {"sec_to_time(c_int_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"sec_to_time(c_bigint_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, @@ -1847,11 +1847,11 @@ func (s *InferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"time_to_sec(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"time_to_sec(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, - {"str_to_date(c_varchar, '%Y:%m:%d')", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDateWidth, int(types.MinFsp)}, - {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, int(types.MinFsp)}, - {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s.%f')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, - {"str_to_date(c_varchar, '%H:%i:%s')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, int(types.MinFsp)}, - {"str_to_date(c_varchar, '%H:%i:%s.%f')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, int(types.MaxFsp)}, + {"str_to_date(c_varchar, '%Y:%m:%d')", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDateWidth, types.MinFsp}, + {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, types.MinFsp}, + {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s.%f')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, + {"str_to_date(c_varchar, '%H:%i:%s')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthNoFsp, types.MinFsp}, + {"str_to_date(c_varchar, '%H:%i:%s.%f')", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDurationWidthWithFsp, types.MaxFsp}, {"period_add(c_int_d , c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, {"period_add(c_bigint_d , c_int_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 6, 0}, @@ -1896,7 +1896,7 @@ func (s *InferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"get_format(DATE, 'USA')", mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.NotNullFlag, 17, types.UnspecifiedLength}, - {"convert_tz(c_time_d, c_text_d, c_text_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, int(types.MaxFsp)}, + {"convert_tz(c_time_d, c_text_d, c_text_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp}, {"convert_tz('2004-01-01 12:00:00.123', c_text_d, c_text_d)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 23, 3}, {"from_unixtime(20170101.999)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 23, 3}, diff --git a/expression/util.go b/expression/util.go index f75775712dce1..6092ac5057b0e 100644 --- a/expression/util.go +++ b/expression/util.go @@ -190,7 +190,7 @@ func ExtractColumnsAndCorColumnsFromExpressions(result []*Column, list []Express } // ExtractColumnSet extracts the different values of `UniqueId` for columns in expressions. -func ExtractColumnSet(exprs []Expression) *intsets.Sparse { +func ExtractColumnSet(exprs ...Expression) *intsets.Sparse { set := &intsets.Sparse{} for _, expr := range exprs { extractColumnSet(expr, set) @@ -754,13 +754,13 @@ func DatumToConstant(d types.Datum, tp byte, flag uint) *Constant { } // ParamMarkerExpression generate a getparam function expression. -func ParamMarkerExpression(ctx sessionctx.Context, v *driver.ParamMarkerExpr) (Expression, error) { +func ParamMarkerExpression(ctx sessionctx.Context, v *driver.ParamMarkerExpr, needParam bool) (*Constant, error) { useCache := ctx.GetSessionVars().StmtCtx.UseCache isPointExec := ctx.GetSessionVars().StmtCtx.PointExec tp := types.NewFieldType(mysql.TypeUnspecified) types.DefaultParamTypeForValue(v.GetValue(), tp) value := &Constant{Value: v.Datum, RetType: tp} - if useCache || isPointExec { + if useCache || isPointExec || needParam { value.ParamMarker = &ParamMarker{ order: v.Order, ctx: ctx, @@ -795,7 +795,7 @@ func PosFromPositionExpr(ctx sessionctx.Context, v *ast.PositionExpr) (int, bool if v.P == nil { return v.N, false, nil } - value, err := ParamMarkerExpression(ctx, v.P.(*driver.ParamMarkerExpr)) + value, err := ParamMarkerExpression(ctx, v.P.(*driver.ParamMarkerExpr), false) if err != nil { return 0, true, err } @@ -1180,11 +1180,7 @@ func (r *SQLDigestTextRetriever) runFetchDigestQuery(ctx context.Context, sctx s stmt += " where digest in (" + strings.Repeat("%?,", len(inValues)-1) + "%?)" } - stmtNode, err := exec.ParseWithParams(ctx, true, stmt, inValues...) - if err != nil { - return nil, err - } - rows, _, err := exec.ExecRestrictedStmt(ctx, stmtNode) + rows, _, err := exec.ExecRestrictedSQL(ctx, nil, stmt, inValues...) if err != nil { return nil, err } diff --git a/go.mod b/go.mod index 32195feaa8268..78cdb578db0bb 100644 --- a/go.mod +++ b/go.mod @@ -31,75 +31,77 @@ require ( github.com/golang/mock v1.6.0 github.com/golang/protobuf v1.5.2 github.com/golang/snappy v0.0.3 - github.com/google/btree v1.0.0 - github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 + github.com/google/btree v1.0.1 + github.com/google/pprof v0.0.0-20211122183932-1daafda22083 github.com/google/uuid v1.1.2 github.com/gorilla/handlers v1.5.1 // indirect github.com/gorilla/mux v1.8.0 - github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 + github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 github.com/jedib0t/go-pretty/v6 v6.2.2 github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef // indirect github.com/opentracing/basictracer-go v1.0.0 - github.com/opentracing/opentracing-go v1.1.0 + github.com/opentracing/opentracing-go v1.2.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20210831093107-2f6cb8008145 - github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 + github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c - github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 + github.com/pingcap/kvproto v0.0.0-20220215045702-d229fcc888c8 + github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 - github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible + github.com/pingcap/tidb-tools v5.4.1-0.20220221041303-00998a9a4bfd+incompatible github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 - github.com/prometheus/client_golang v1.5.1 + github.com/pingcap/tipb v0.0.0-20220110031732-29e23c62eeac + github.com/prometheus/client_golang v1.11.0 github.com/prometheus/client_model v0.2.0 - github.com/prometheus/common v0.9.1 + github.com/prometheus/common v0.26.0 github.com/shirou/gopsutil/v3 v3.21.12 + github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 // indirect github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 + github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546 // indirect github.com/sirupsen/logrus v1.8.1 // indirect - github.com/soheilhy/cmux v0.1.4 - github.com/spf13/cobra v1.0.0 + github.com/soheilhy/cmux v0.1.5 + github.com/spf13/cobra v1.1.3 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20211229051614-62d6b4a2e8f7 - github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee + github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5 + github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible github.com/uber/jaeger-lib v2.4.1+incompatible // indirect github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0 - go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 + go.etcd.io/etcd/api/v3 v3.5.2 + go.etcd.io/etcd/client/pkg/v3 v3.5.2 + go.etcd.io/etcd/client/v3 v3.5.2 + go.etcd.io/etcd/server/v3 v3.5.2 + go.etcd.io/etcd/tests/v3 v3.5.2 go.uber.org/atomic v1.9.0 go.uber.org/automaxprocs v1.4.0 go.uber.org/goleak v1.1.12 go.uber.org/multierr v1.7.0 - go.uber.org/zap v1.19.1 + go.uber.org/zap v1.20.0 golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e golang.org/x/text v0.3.7 - golang.org/x/time v0.0.0-20191024005414-555d28b269f0 + golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba golang.org/x/tools v0.1.8 google.golang.org/api v0.54.0 - google.golang.org/grpc v1.40.0 + google.golang.org/grpc v1.43.0 gopkg.in/yaml.v2 v2.4.0 modernc.org/mathutil v1.4.1 sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -// cloud.google.com/go/storage will upgrade grpc to v1.40.0 -// we need keep the replacement until go.etcd.io supports the higher version of grpc. -replace google.golang.org/grpc => google.golang.org/grpc v1.29.1 - replace github.com/pingcap/tidb/parser => ./parser // fix potential security issue(CVE-2020-26160) introduced by indirect dependency. diff --git a/go.sum b/go.sum index e6e14649c1d61..33a2877185b9e 100644 --- a/go.sum +++ b/go.sum @@ -1,3 +1,4 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= @@ -31,6 +32,7 @@ cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4g cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -67,37 +69,32 @@ github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= -github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= -github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= -github.com/ReneKroon/ttlcache/v2 v2.3.0/go.mod h1:zbo6Pv/28e21Z8CzzqgYRArQYGYtjONRxaAKGxzQvG4= github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= -github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= github.com/VividCortex/ewma v1.1.1 h1:MnEK4VOv6n0RSY4vtRe3h11qjxL3+t0B8yOL8iMXdcM= github.com/VividCortex/ewma v1.1.1/go.mod h1:2Tkkvm3sRDVXaiyucHiACn4cqf7DpdyLvmxzcbUokwA= -github.com/VividCortex/mysqlerr v0.0.0-20200629151747-c28746d985dd/go.mod h1:f3HiCrHjHBdcm6E83vGaXh1KomZMA2P6aeo3hKx/wg0= -github.com/Xeoncross/go-aesctr-with-hmac v0.0.0-20200623134604-12b17a7ff502/go.mod h1:pmnBM9bxWSiHvC/gSWunUIyDvGn33EkP2CUjxFKtTTM= github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= -github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT0m65DJ+Wo= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/thrift v0.0.0-20181112125854-24918abba929/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714 h1:Jz3KVLYY5+JO7rDiX0sAuRGtuv2vG01r17Y9nLMWNUw= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= -github.com/appleboy/gin-jwt/v2 v2.6.3/go.mod h1:MfPYA4ogzvOcVkRwAxT7quHOtQmVKDpTwxyUrC2DNw0= -github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go v1.35.3 h1:r0puXncSaAfRt7Btml2swUo74Kao+vKhO3VLjwDjK54= github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= +github.com/benbjohnson/clock v1.0.3/go.mod h1:bGMdMPoPVvcYyt1gHDf4J2KE153Yf9BuiUKYMaxlTDM= github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= @@ -105,14 +102,14 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= -github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= -github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5/go.mod h1:jtAfVaU/2cu1+wdSRPWE2c1N2qeAA3K4RH9pYgqwets= github.com/carlmjohnson/flagext v0.21.0 h1:/c4uK3ie786Z7caXLcIMvePNSSiH3bQVGDvmGLMme60= github.com/carlmjohnson/flagext v0.21.0/go.mod h1:Eenv0epIUAr4NuedNmkzI8WmBmjIxZC239XcKxYS2ac= -github.com/cenkalti/backoff/v4 v4.0.2/go.mod h1:eEew/i+1Q6OrCDZh3WiXYv3+nJwBASZ8Bog/87DQnVg= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/certifi/gocertifi v0.0.0-20191021191039-0944d244cd40/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= +github.com/certifi/gocertifi v0.0.0-20200922220541-2c3bb06c6054/go.mod h1:sGbDF6GwGcLpkNXPUTkMRoywsNa/ol15pxFe6ERfguA= github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= @@ -124,10 +121,19 @@ github.com/cheynewallace/tabby v1.1.1/go.mod h1:Pba/6cUL8uYqvOc9RkyvFbHGrQ9wShyr github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= -github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= +github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cockroachdb/datadriven v0.0.0-20200714090401-bf6692d28da5/go.mod h1:h6jFvWxBdQXxjopDMZyH2UVceIRfR84bdzbkoKrsWNo= github.com/cockroachdb/datadriven v1.0.0 h1:uhZrAfEayBecH2w2tZmhe20HJ7hDvrrA4x2Bg9YdZKM= github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/errors v1.2.4/go.mod h1:rQD95gz6FARkaKkQXUksEje/d9a6wBJoCr5oaCLELYA= github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= @@ -150,21 +156,18 @@ github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2 github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e h1:Wf6HqHfScWJN9/ZjdUKyjop4mf3Qdd+1TvvltAvM3m8= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbpBpLoyyu8B6e44T7hJy6potg= +github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= +github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= -github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawkhRnnX0D1bvVI= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso= @@ -189,29 +192,32 @@ github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= -github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385 h1:clC1lXBpe2kTj2VHdaIu9ajZQe4kcEY9j0NsnDDBZ3o= github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= +github.com/envoyproxy/go-control-plane v0.9.9-0.20201210154907-fd9021fe5dad/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= +github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/etcd-io/gofail v0.0.0-20190801230047-ad7f989257ca/go.mod h1:49H/RkXP8pKaZy4h0d+NW16rSLhyVBt4o6VLJbmOqDE= github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= -github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/felixge/httpsnoop v1.0.1 h1:lvB5Jl89CsZtGIWuTcDM1E/vkVs49/Ml7JJe07l8SPQ= github.com/felixge/httpsnoop v1.0.1/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= -github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= +github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/form3tech-oss/jwt-go v3.2.5+incompatible h1:/l4kBbb4/vGSsdtB5nUe8L7B9mImVMaBPw9L/0TBHU8= github.com/form3tech-oss/jwt-go v3.2.5+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= -github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible h1:0sWoh2EtO7UrQdNTAN+hnU3QXa4AoivplyPLLHkcrLk= github.com/form3tech-oss/jwt-go v3.2.6-0.20210809144907-32ab6a8243d7+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= @@ -220,18 +226,12 @@ github.com/fsouza/fake-gcs-server v1.19.0 h1:XyaGOlqo+R5sjT03x2ymk0xepaQlgwhRLTT github.com/fsouza/fake-gcs-server v1.19.0/go.mod h1:JtXHY/QzHhtyIxsNfIuQ+XgHtRb5B/w8nqbL5O8zqo0= github.com/fzipp/gocyclo v0.3.1/go.mod h1:DJHO6AUmbdqj2ET4Z9iArSuwWgYDRryYt2wASxc7x3E= github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= +github.com/getsentry/raven-go v0.2.0/go.mod h1:KungGk8q33+aIAZUIVWZDr2OfAEBsO49PX4NzFV5kcQ= github.com/ghemawat/stream v0.0.0-20171120220530-696b145b53b9/go.mod h1:106OIgooyS7OzLDOpUGgm9fA3bQENb/cFSyyBmMoJDs= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= -github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= -github.com/gin-contrib/sse v0.0.0-20170109093832-22d885f9ecc7/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= -github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= -github.com/gin-gonic/gin v1.3.0/go.mod h1:7cKuhb5qV2ggCFctp2fJQ+ErvciLZrIeoOSOm6mUr7Y= github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= -github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= -github.com/go-chi/chi v4.0.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= -github.com/go-echarts/go-echarts v1.0.0/go.mod h1:qbmyAb/Rl1f2w7wKba1D4LoNq4U164yO4/wedFbcWyo= github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= @@ -239,37 +239,22 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-ole/go-ole v1.2.6 h1:/Fpf6oFPoeFik9ty7siob0G6Ke8QvQEuVcuChpwXzpY= github.com/go-ole/go-ole v1.2.6/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= -github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= -github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= -github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= -github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= -github.com/go-openapi/jsonreference v0.19.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= -github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= -github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= -github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= -github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= -github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= -github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= -github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= -github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= -github.com/go-resty/resty/v2 v2.6.0/go.mod h1:PwvJS6hvaPkjtjNg9ph+VrSD92bi5Zq73w/BIH7cC3Q= github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= -github.com/goccy/go-graphviz v0.0.5/go.mod h1:wXVsXxmyMQU6TN3zGRttjNn3h+iCAS7xQFC6TlNvLhk= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= github.com/gogo/protobuf v0.0.0-20171007142547-342cbe0a0415/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -282,12 +267,12 @@ github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69 github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= -github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -325,8 +310,9 @@ github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4= +github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -353,21 +339,19 @@ github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OI github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= -github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 h1:K6RDEckDVWvDI9JAJYCmNdQXq6neHJOYx3V6jnqNEec= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20211122183932-1daafda22083 h1:c8EUapQFi+kjzedr4c6WqbwMdmB95+oDBWZ5XFHFYxY= +github.com/google/pprof v0.0.0-20211122183932-1daafda22083/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= -github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= -github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2 h1:EVhdT+1Kseyi1/pUmXKaFxYsDNy9RQYkMWRH68J/W7Y= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -377,40 +361,54 @@ github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORR github.com/gorilla/handlers v1.4.2/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= -github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.7.4/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= -github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdRqNr0QAUJTIkQAUtFjg= github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= -github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= +github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= +github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= +github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 h1:VHgatEHNcBFEB7inlalqfNqw65aNkM1lGX2yt3NmbS8= github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334/go.mod h1:SK73tn/9oHe+/Y0h39VT4UCxmurVJkR5NA7kMEAOgSE= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= -github.com/influxdata/tdigest v0.0.1/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= @@ -418,35 +416,30 @@ github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrO github.com/jcmturner/gofork v0.0.0-20180107083740-2aebee971930/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= github.com/jedib0t/go-pretty/v6 v6.2.2 h1:o3McN0rQ4X+IU+HduppSp9TwRdGLRW2rhJXy9CJaCRw= github.com/jedib0t/go-pretty/v6 v6.2.2/go.mod h1:+nE9fyyHGil+PuISTCrp7avEdo6bqoMwqZnuiK2r2a0= -github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= -github.com/jinzhu/inflection v1.0.0/go.mod h1:h+uFLlag+Qp1Va5pdKtLDYj+kHp5pxUVkryuEj+Srlc= -github.com/jinzhu/now v1.1.1/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= -github.com/jinzhu/now v1.1.2/go.mod h1:d3SSVoowX0Lcu0IBviAWJpolVfI5UJVZZ7cO71lE/z8= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= -github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df h1:Zrb0IbuLOGHL7nrO2WrcuNWgDTlzFv3zY69QMx4ggQE= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df/go.mod h1:mAVCUAYtW9NG31eB30umMSLKcDt6mCUWSjoSn5qBh0k= -github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= -github.com/joomcode/errorx v1.0.1/go.mod h1:kgco15ekB6cs+4Xjzo7SPeXzx38PbJzBwbnu9qfVNHQ= -github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= +github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= -github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= -github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= -github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= -github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= @@ -473,48 +466,42 @@ github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORN github.com/kr/pretty v0.2.1 h1:Fmg33tUaq4/8ym9TJN1x7sLJnHVwhP33CNkpYV/7rwI= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= -github.com/kr/pty v1.1.5/go.mod h1:9r2w37qlBe7rQ6e1fg1S/9xpWHSnaqNdHD3WcMdbPDA= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= -github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0/go.mod h1:zJYVVT2jmtg6P3p1VtQj7WsuWi/y4VnjVBn7F8KPB3I= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= +github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= -github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.8 h1:c1ghPdyEDarC70ftn0y+A/Ee++9zz8ljHG1b13eJ0s8= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= -github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= +github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= -github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= -github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.12 h1:Y41i/hVW3Pgwr8gV+J23B9YEY0zxjptBuCWEaxmAOow= github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= -github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y= -github.com/mattn/go-sqlite3 v1.14.5/go.mod h1:WVKg1VTActs4Qso6iwGbiFih2UIHo0ENGwNd0Lj+XmI= github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= -github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= -github.com/minio/sio v0.3.0/go.mod h1:8b0yPp2avGThviy/+OCJBI6OMpvxoUuiLvE6F1lebhw= +github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= +github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= +github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= +github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= +github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= @@ -523,70 +510,59 @@ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lN github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modocache/gover v0.0.0-20171022184752-b58185e213c5/go.mod h1:caMODM3PzxT8aQXRPkAt8xlV/e7d7w8GM5g0fa5F0D8= -github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= -github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/ncw/directio v1.0.4 h1:CojwI07mCEmRkajgx42Pf8jyCwTs1ji9/Ij9/PJG12k= github.com/ncw/directio v1.0.4/go.mod h1:CKGdcN7StAaqjT7Qack3lAXeX4pjnyc46YeqZH1yWVY= -github.com/nfnt/resize v0.0.0-20160724205520-891127d8d1b5/go.mod h1:jpp1/29i3P1S/RLdc7JQKbRpFeM1dOBd8T9ki5s+AY8= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdcNTgsos+vFzULLwyElndwn+5c= github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= -github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= -github.com/oleiade/reflections v1.0.1/go.mod h1:rdFxbxq4QXVZWj0F+e9jqjDkc7dbp97vkRixKo2JR60= -github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= +github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= -github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= -github.com/onsi/ginkgo v1.13.0 h1:M76yO2HkZASFjXL0HSoZJ1AYEmQxNJmY41Jx1zNUq1Y= github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= -github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/ginkgo/v2 v2.0.0/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= -github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= +github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= +github.com/onsi/gomega v1.18.1/go.mod h1:0q+aL8jAiMXy9hbwj2mr5GziHiwhAIQpFmmtT5hitRs= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= -github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= +github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= +github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= -github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 h1:JhzVVoYvbOACxoUmOs6V/G4D5nPVUW73rKvXxP4XUJc= github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2/go.mod h1:iIss55rKnNBTvrwdmkUpLnDpZoAHvWaiq5+iMmen4AE= -github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= -github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20210831093107-2f6cb8008145 h1:t7sdxmfyZ3p9K7gD8t5B50TerzTvHuAPYt+VubTVKDY= github.com/pingcap/badger v1.5.1-0.20210831093107-2f6cb8008145/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 h1:R8gStypOBmpnHEx1qi//SaqxJVI4inOqljg/Aj5/390= -github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= -github.com/pingcap/errcode v0.3.0 h1:IF6LC/4+b1KNwrMlr2rBTUrojFPMexXBcDWZSpNwxjg= -github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIzHvz6SeM= +github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0 h1:HVl5539r48eA+uDuX/ziBmQCxzT1pGrzWbKuXT46Bq0= +github.com/pingcap/check v0.0.0-20211026125417-57bd13f7b5f0/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20200917111840-a15ef68f753d/go.mod h1:g4vx//d6VakjJ0mk7iLBlKA8LFavV/sAVINT/1PFxeQ= -github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c h1:xpW9bvK+HuuTmyFqUwr+jcCvpVkK7sumiz+ko5H9eq4= github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= -github.com/pingcap/failpoint v0.0.0-20191029060244-12f4ac2fd11d/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= -github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= -github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd/go.mod h1:IVF+ijPSMZVtx2oIqxAg7ur6EyixtTYfOHwpfmlhqI4= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00 h1:C3N3itkduZXDZFh4N3vQ5HEtld3S+Y+StULhWVvumU0= github.com/pingcap/failpoint v0.0.0-20210918120811-547c13e3eb00/go.mod h1:4qGtCB0QK0wBzKtFEGDhxXnSnbQApw1gc9siScUl8ew= github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 h1:Pe2LbxRmbTfAoKJ65bZLmhahmvHm7n9DUxGRQT00208= @@ -594,28 +570,20 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210819164333-bd5706b9d9f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211109071446-a8b4d34474bc/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211122024046-03abd340988f/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c h1:4D/M5eYfbswv3vs0ZtbVgNKwSRMXgAcm+9a+IbC7q0o= -github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220106070556-3fa8fa04f898/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220215045702-d229fcc888c8 h1:8VizThgCQJTVwCrfp5JOsiUdpvB8F9nsUcLrnHqjpNY= +github.com/pingcap/kvproto v0.0.0-20220215045702-d229fcc888c8/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= -github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 h1:SvWCbCPh1YeHd9yQLksvJYAgft6wLTY1aNG81tpyscQ= -github.com/pingcap/log v0.0.0-20210906054005-afc726e70354/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= -github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= -github.com/pingcap/sysutil v0.0.0-20210730114356-fcd8a63f68c5/go.mod h1:XsOaV712rUk63aOEKYP9PhXTIE3FMNHmC2r1wX5wElY= +github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee h1:VO2t6IBpfvW34TdtD/G10VvnGqjLic1jzOuHjUb5VqM= +github.com/pingcap/log v0.0.0-20211215031037-e024ba4eb0ee/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= -github.com/pingcap/tidb-dashboard v0.0.0-20211008050453-a25c25809529/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-dashboard v0.0.0-20211107164327-80363dfbe884/go.mod h1:OCXbZTBTIMRcIt0jFsuCakZP+goYRv6IjawKbwLS2TQ= -github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible h1:c7+izmker91NkjkZ6FgTlmD4k1A5FLOAq+li6Ki2/GY= -github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 h1:DHU4vw0o15qdKsf7d/Pyhun4YtX8FwoDQxG0plPByUg= -github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tidb-tools v5.4.1-0.20220221041303-00998a9a4bfd+incompatible h1:t5NkebXbboNFUh8Z/yOvezIxdx7rt1/FSh5jBj6mmjI= +github.com/pingcap/tidb-tools v5.4.1-0.20220221041303-00998a9a4bfd+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tipb v0.0.0-20220110031732-29e23c62eeac h1:bVklq/Np5uHUylW4Htyhi92TNrvIqVwht/+bHk0R/Tk= +github.com/pingcap/tipb v0.0.0-20220110031732-29e23c62eeac/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -625,15 +593,16 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= -github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= -github.com/prometheus/client_golang v1.5.1 h1:bdHYieyGlH+6OLEk2YQha8THib30KP0/yD0YH9m6xcA= -github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.0 h1:HNkLOAEQMIDv/K+04rukrLx6ch7msSRwf3/SASFAGtQ= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -643,16 +612,15 @@ github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39/go.mod h1:daVV7q github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= -github.com/prometheus/common v0.9.1 h1:KOMtN28tlbam3/7ZKEYKHhKoJZYYj3gMH4uc62x7X7U= -github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= -github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= -github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= @@ -662,92 +630,71 @@ github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJ github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= -github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= -github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= -github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= -github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil/v3 v3.21.12 h1:VoGxEW2hpmz0Vt3wUvHIl9fquzYLNpVpgNNB7pGJimA= github.com/shirou/gopsutil/v3 v3.21.12/go.mod h1:BToYZVTlSVlfazpDDYFnsVZLaoRG+g8ufT6fPQLdJzA= -github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749 h1:bUGsEnyNbVPw06Bs80sCeARAlK8lhwqGyi6UT8ymuGk= github.com/shurcooL/httpfs v0.0.0-20190707220628-8d4bc4ba7749/go.mod h1:ZY1cvUeJuFPAdZ/B6v7RHavJWZn2YPVFQ1OSXhCGOkg= github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 h1:mj/nMDAwTBiaCqMEs4cYCqF7pO6Np7vhy1D1wcQGz+E= github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0/go.mod h1:919LwcH0M7/W4fcZ0/jy0qGght1GIhqyS/EgWGH2j5Q= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd h1:ug7PpSOB5RBPK1Kg6qskGBoP3Vnj/aNYFTznWvlkGo0= -github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= +github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546 h1:pXY9qYc/MP5zdvqWEUH6SjNiu7VhSjuVFTFiTcphaLU= +github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= -github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= -github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= +github.com/spf13/cobra v1.1.3 h1:xghbfqPkxzxP3C/f3n5DdpAbdKLj4ZE4BWQI362l53M= +github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= -github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= -github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= +github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= -github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= -github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14/go.mod h1:gxQT6pBGRuIGunNf/+tSOB5OHvguWi8Tbt82WOkf35E= -github.com/swaggo/gin-swagger v1.2.0/go.mod h1:qlH2+W7zXGZkczuL+r2nEBR2JTT+/lX05Nn6vPhc7OI= -github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lAbCgAAX/KZ80LM/OXwtWFI/5TvZlwxSg8Cq08PV0= -github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+tv8Y= -github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= -github.com/swaggo/swag v1.6.6-0.20200529100950-7c765ddd0476/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= -github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965 h1:1oFLiOyVl+W7bnBzGhf7BbIv9loSFQcieWWYIjLqcAw= -github.com/syndtr/goleveldb v1.0.1-0.20190318030020-c3a204f8e965/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= -github.com/thoas/go-funk v0.8.0/go.mod h1:+IWnUfUmFO1+WVYQWQtIJHeRRdaIyyYglZN7xzUPe4Q= +github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= -github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= -github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211229051614-62d6b4a2e8f7 h1:megVHSEVsedArz10MwJTlrTtBNhORupPbuVAUhHbSko= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20211229051614-62d6b4a2e8f7/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= -github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= -github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= -github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= -github.com/tklauser/go-sysconf v0.3.4/go.mod h1:Cl2c8ZRWfHD5IrfHo9VN+FX9kCFjIOyVklgXycLB6ek= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5 h1:xfhhUvnueHqt+mYfRR5ONAVezrqNBCvcWSh4Ml6Isjc= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220217140116-589cf79b2ab5/go.mod h1:gaHSp8rnxZ0w36qb6QPPNPh9P0Mu5vAEwCQcc0Brni4= +github.com/tikv/pd/client v0.0.0-20220216070739-26c668271201/go.mod h1:fEvI5fhAuJn1Fn87VJF8ByE9Vc16EzWGoePZB21/nL8= +github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7 h1:RRiYmyzHgTgV0mrp6Ue6cGn0EAB7U1YHnEfVWEGCKk8= +github.com/tikv/pd/client v0.0.0-20220216080339-1b8f82378ee7/go.mod h1:fEvI5fhAuJn1Fn87VJF8ByE9Vc16EzWGoePZB21/nL8= github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= -github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= github.com/tklauser/numcpus v0.3.0 h1:ILuRUQBtssgnxw0XXIjKUC56fgnOrFoQQ/4+DeU2biQ= github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8= -github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= -github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= @@ -755,26 +702,12 @@ github.com/uber/jaeger-client-go v2.22.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMW github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= -github.com/ugorji/go v1.1.5-pre/go.mod h1:FwP/aQVg39TXzItUBMwnWp9T9gPQnXw4Poh4/oBQZ/0= -github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= -github.com/ugorji/go/codec v0.0.0-20181022190402-e5e69e061d4f/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= -github.com/ugorji/go/codec v1.1.5-pre/go.mod h1:tULtS6Gy1AE1yCENaw4Vb//HLH5njI2tfCQDUqRd8fI= -github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= -github.com/unrolled/render v1.0.1 h1:VDDnQQVfBMsOsp3VaCJszSO0nkBIVEYoPWeRThk9spY= -github.com/unrolled/render v1.0.1/go.mod h1:gN9T0NhL4Bfbwu8ann7Ry/TGHYfosul+J0obPf6NBdM= -github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= -github.com/urfave/cli/v2 v2.1.1/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ= -github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/urfave/negroni v1.0.0 h1:kIimOitoypq34K7TG7DUaJ9kq/N4Ofuwi1sjz0KipXc= github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= -github.com/vmihailenco/msgpack/v5 v5.0.0-beta.1/go.mod h1:xlngVLeyQ/Qi05oQxhQ+oTuqa03RjMwMfk/7/TCs+QI= -github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= @@ -790,7 +723,6 @@ github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0 h1:a74 github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0/go.mod h1:HYhIKsdns7xz80OgkbgJYrtQY7FjHWHKH6cvN7+czGE= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= -github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= @@ -803,14 +735,25 @@ github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1 github.com/yusufpapurcu/wmi v1.2.2 h1:KBNDSne4vP5mbSWnJbO+51IMOXJB67QiYCSBrubbPRg= github.com/yusufpapurcu/wmi v1.2.2/go.mod h1:SBZ9tNy3G9/m5Oi98Zks0QjeHVDvuK0qfxQmPyzfmi0= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= -go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= -go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 h1:MNsY1TIsWLNCMT4DzZjFOxbDKfSoULYP0OFjJ8dSxts= -go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9/go.mod h1:q+i20RPAmay+xq8LJ3VMOhXCNk4YCk3V7QP91meFavw= +go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= +go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= +go.etcd.io/etcd/api/v3 v3.5.2 h1:tXok5yLlKyuQ/SXSjtqHc4uzNaMqZi2XsoSPr/LlJXI= +go.etcd.io/etcd/api/v3 v3.5.2/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= +go.etcd.io/etcd/client/pkg/v3 v3.5.2 h1:4hzqQ6hIb3blLyQ8usCU4h3NghkqcsohEQ3o3VetYxE= +go.etcd.io/etcd/client/pkg/v3 v3.5.2/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= +go.etcd.io/etcd/client/v2 v2.305.2 h1:ymrVwTkefuqA/rPkSW7/B4ApijbPVefRumkY+stNfS0= +go.etcd.io/etcd/client/v2 v2.305.2/go.mod h1:2D7ZejHVMIfog1221iLSYlQRzrtECw3kz4I4VAQm3qI= +go.etcd.io/etcd/client/v3 v3.5.2 h1:WdnejrUtQC4nCxK0/dLTMqKOB+U5TP/2Ya0BJL+1otA= +go.etcd.io/etcd/client/v3 v3.5.2/go.mod h1:kOOaWFFgHygyT0WlSmL8TJiXmMysO/nNUlEsSsN6W4o= +go.etcd.io/etcd/etcdutl/v3 v3.5.2/go.mod h1:f+KEUNxRzqQGq1Y/SsaDN5cmlOGRWgfE3lXEDi5F1Ys= +go.etcd.io/etcd/pkg/v3 v3.5.2 h1:YZUojdoPhOyl5QILYnR8LTUbbNefu/sV4ma+ZMr2tto= +go.etcd.io/etcd/pkg/v3 v3.5.2/go.mod h1:zsXz+9D/kijzRiG/UnFGDTyHKcVp0orwiO8iMLAi+k0= +go.etcd.io/etcd/raft/v3 v3.5.2 h1:uCC37qOXqBvKqTGHGyhASsaCsnTuJugl1GvneJNwHWo= +go.etcd.io/etcd/raft/v3 v3.5.2/go.mod h1:G6pCP1sFgbjod7/KnEHY0vHUViqxjkdt6AiKsD0GRr8= +go.etcd.io/etcd/server/v3 v3.5.2 h1:B6ytJvS4Fmt8nkjzS2/8POf4tuPhFMluE0lWd4dx/7U= +go.etcd.io/etcd/server/v3 v3.5.2/go.mod h1:mlG8znIEz4N/28GABrohZCBM11FqgGVQcpbcyJgh0j0= +go.etcd.io/etcd/tests/v3 v3.5.2 h1:uk7/uMGVebpBDl+roivowHt6gJ5Fnqwik3syDkoSKdo= +go.etcd.io/etcd/tests/v3 v3.5.2/go.mod h1:Jdzbei4uFi9C3xDBfCwckRXjlX0UPooiP4g/zXgBMgQ= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= @@ -819,6 +762,28 @@ go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0 h1:gqCw0LfLxScz8irSi8exQc7fyQ0fKQU/qnC/X8+V/1M= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= +go.opentelemetry.io/contrib v0.20.0 h1:ubFQUn0VCZ0gPwIoJfBJVpeBlyRMxu8Mm/huKWYd9p0= +go.opentelemetry.io/contrib v0.20.0/go.mod h1:G/EtFaa6qaN7+LxqfIAT3GiZa7Wv5DTBUzl5H4LY0Kc= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0 h1:sO4WKdPAudZGKPcpZT4MJn6JaDmpyLrMPDGGyA1SttE= +go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.20.0/go.mod h1:oVGt1LRbBOBq1A5BQLlUg9UaU/54aiHw8cgjV3aWZ/E= +go.opentelemetry.io/otel v0.20.0 h1:eaP0Fqu7SXHwvjiqDq83zImeehOHX8doTvU9AwXON8g= +go.opentelemetry.io/otel v0.20.0/go.mod h1:Y3ugLH2oa81t5QO+Lty+zXf8zC9L26ax4Nzoxm/dooo= +go.opentelemetry.io/otel/exporters/otlp v0.20.0 h1:PTNgq9MRmQqqJY0REVbZFvwkYOA85vbdQU/nVfxDyqg= +go.opentelemetry.io/otel/exporters/otlp v0.20.0/go.mod h1:YIieizyaN77rtLJra0buKiNBOm9XQfkPEKBeuhoMwAM= +go.opentelemetry.io/otel/metric v0.20.0 h1:4kzhXFP+btKm4jwxpjIqjs41A7MakRFUS86bqLHTIw8= +go.opentelemetry.io/otel/metric v0.20.0/go.mod h1:598I5tYlH1vzBjn+BTuhzTCSb/9debfNp6R3s7Pr1eU= +go.opentelemetry.io/otel/oteltest v0.20.0 h1:HiITxCawalo5vQzdHfKeZurV8x7ljcqAgiWzF6Vaeaw= +go.opentelemetry.io/otel/oteltest v0.20.0/go.mod h1:L7bgKf9ZB7qCwT9Up7i9/pn0PWIa9FqQ2IQ8LoxiGnw= +go.opentelemetry.io/otel/sdk v0.20.0 h1:JsxtGXd06J8jrnya7fdI/U/MR6yXA5DtbZy+qoHQlr8= +go.opentelemetry.io/otel/sdk v0.20.0/go.mod h1:g/IcepuwNsoiX5Byy2nNV0ySUF1em498m7hBWC279Yc= +go.opentelemetry.io/otel/sdk/export/metric v0.20.0 h1:c5VRjxCXdQlx1HjzwGdQHzZaVI82b5EbBgOu2ljD92g= +go.opentelemetry.io/otel/sdk/export/metric v0.20.0/go.mod h1:h7RBNMsDJ5pmI1zExLi+bJK+Dr8NQCh0qGhm1KDnNlE= +go.opentelemetry.io/otel/sdk/metric v0.20.0 h1:7ao1wpzHRVKf0OQ7GIxiQJA6X7DLX9o14gmVon7mMK8= +go.opentelemetry.io/otel/sdk/metric v0.20.0/go.mod h1:knxiS8Xd4E/N+ZqKmUPf3gTTZ4/0TjTXukfxjzSTpHE= +go.opentelemetry.io/otel/trace v0.20.0 h1:1DL6EXUdcg95gukhuRRvLDO/4X5THh/5dIV52lqtnbw= +go.opentelemetry.io/otel/trace v0.20.0/go.mod h1:6GjCW8zgDjwGHGa6GkyeB8+/5vjT16gUEi0Nf1iBdgw= +go.opentelemetry.io/proto/otlp v0.7.0 h1:rwOQPCuKAKmwGKq2aVNnYIibI6wnV7EvzgfTCzcdGg8= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= @@ -828,17 +793,13 @@ go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/automaxprocs v1.4.0 h1:CpDZl6aOlLhReez+8S3eEotD7Jx0Os++lemPlMULQP0= go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q= -go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= -go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= -go.uber.org/goleak v0.10.0/go.mod h1:VCZuO8V8mFPlL0F5J5GK1rtHV3DrFcQ1R8ryq7FK0aI= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= -go.uber.org/goleak v1.1.11-0.20210813005559-691160354723/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= -go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= @@ -846,25 +807,22 @@ go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9E go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.12.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM= -go.uber.org/zap v1.15.0/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc= -go.uber.org/zap v1.16.0/go.mod h1:MA8QOfq0BHJwdXa996Y4dYkAqRKB8/1K1QMMZVaNZjQ= +go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= go.uber.org/zap v1.18.1/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= go.uber.org/zap v1.19.0/go.mod h1:xg/QME4nWcxGxrpdeYfq7UvYrLh66cuVKdrbD1XF/NI= -go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= -go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= +go.uber.org/zap v1.20.0 h1:N4oPlghZwYG55MlU6LXk/Zp00FVNE9X9wrYO8CEs4lc= +go.uber.org/zap v1.20.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190513172903-22d7a77e9e5f/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20190611184440-5c40567a22f8/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= -golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897 h1:pLI5jrR7OSLijeIDcmRxNmw2api+jEfxLoykJVice/E= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -886,7 +844,7 @@ golang.org/x/exp v0.0.0-20200513190911-00229845015e/go.mod h1:4M0jN8W1tt0AVLNr8H golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= -golang.org/x/image v0.0.0-20200119044424-58c23975cae1/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= @@ -912,9 +870,12 @@ golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -923,14 +884,11 @@ golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= -golang.org/x/net v0.0.0-20190522155817-f3200d17e092/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= -golang.org/x/net v0.0.0-20190611141213-3f473d35a33a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= -golang.org/x/net v0.0.0-20190813141303-74dc4d7220e7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20191002035440-2ec189313ef0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= @@ -952,11 +910,13 @@ golang.org/x/net v0.0.0-20201010224723-4f7140c49acb/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= +golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= @@ -990,12 +950,14 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20181228144115-9a3f9b0469bb/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1004,14 +966,10 @@ golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190610200419-93c9922d18ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1020,6 +978,7 @@ golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1034,15 +993,19 @@ golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1052,12 +1015,13 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211013075003-97ac67df715c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e h1:fLOSk5Q00efkSvAm+4xcoXD+RRmLmmulPn5I3Y9F2EM= @@ -1073,16 +1037,17 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= -golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= @@ -1093,11 +1058,7 @@ golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3 golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190524210228-3d17549cdc6b/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= -golang.org/x/tools v0.0.0-20190606050223-4d9ae51c2468/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190611222205-d73e1c7e250b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= -golang.org/x/tools v0.0.0-20190614205625-5aca471b1d59/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1105,11 +1066,10 @@ golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1123,7 +1083,6 @@ golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= -golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= @@ -1141,9 +1100,9 @@ golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210112230658-8b4aab62c064/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= @@ -1158,9 +1117,7 @@ golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= -gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= -gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= @@ -1190,6 +1147,7 @@ google.golang.org/api v0.50.0/go.mod h1:4bNT5pAuq5ji4SRZm+5QIkjny9JAyVD/3gaSihNe google.golang.org/api v0.51.0/go.mod h1:t4HdrdoNgyN5cbEfm7Lum0lcLDLiise1F8qDKX00sOU= google.golang.org/api v0.54.0 h1:ECJUVngj71QI6XEm7b1sAf8BljU5inEhMbKPR8Lxhhk= google.golang.org/api v0.54.0/go.mod h1:7C4bFFOvVDGXjfDTAsgGwDgAxRDeQ4X8NvUedIt6z3k= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= @@ -1221,8 +1179,10 @@ google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= @@ -1251,8 +1211,36 @@ google.golang.org/genproto v0.0.0-20210805201207-89edb61ffb67/go.mod h1:ob2IJxKr google.golang.org/genproto v0.0.0-20210813162853-db860fec028c/go.mod h1:cFeNkxwySK631ADgubI+/XFU/xp8FD5KIVV4rj8UC5w= google.golang.org/genproto v0.0.0-20210825212027-de86158e7fda h1:iT5uhT54PtbqUsWddv/nnEWdE5e/MTr+Nv3vjxlBP1A= google.golang.org/genproto v0.0.0-20210825212027-de86158e7fda/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= -google.golang.org/grpc v1.29.1 h1:EC2SB8S04d2r73uptxphDSUG+kTKVgjRPF+N3xpxRB4= +google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.24.0/go.mod h1:XDChyiUovWa60DnaeDeZmSW86xtLtjtZbwvSiRnRtcA= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.1/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.34.0/go.mod h1:WotjhfgOW/POjDeRt8vscBtXq+2VjORFy659qA51WJ8= +google.golang.org/grpc v1.35.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.36.0/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.36.1/go.mod h1:qjiiYl8FncCW8feJPdyg3v6XW24KsRHe+dy9BAGRRjU= +google.golang.org/grpc v1.37.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.37.1/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.39.0/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= +google.golang.org/grpc v1.39.1/go.mod h1:PImNr+rS9TWYb2O4/emRugxiyHZ5JyHW5F+RPnDzfrE= +google.golang.org/grpc v1.40.0/go.mod h1:ogyxbiOoUXAkP+4+xa6PZSE9DZgIHtSpzjDTB9KAK34= +google.golang.org/grpc v1.43.0 h1:Eeu7bZtDZ2DpRCsLhUlcrLnvYaMK1Gz86a+hMVvELmM= +google.golang.org/grpc v1.43.0/go.mod h1:k+4IHHFw41K8+bbowsex27ge2rCb65oeWqe4jJ590SU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= @@ -1268,21 +1256,18 @@ google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp0 google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1 h1:SnqbnDw1V7RiZcXPx5MEeqPv2s79L9i7BJUlG/+RurQ= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= -gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= -gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= -gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= +gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= @@ -1300,7 +1285,6 @@ gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -gopkg.in/yaml.v2 v2.2.7/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= @@ -1308,10 +1292,6 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gorm.io/driver/mysql v1.0.6/go.mod h1:KdrTanmfLPPyAOeYGyG+UpDys7/7eeWT1zCq+oekYnU= -gorm.io/driver/sqlite v1.1.4/go.mod h1:mJCeTFr7+crvS+TRnWc5Z3UvwxUN1BGBLMrf5LA9DYw= -gorm.io/gorm v1.20.7/go.mod h1:0HFTzE/SqkGTzK6TlDPPQbAYCluiVvhzoA1+aVyzenw= -gorm.io/gorm v1.21.9/go.mod h1:F+OptMscr0P2F2qU97WT1WimdH9GaQPoDW7AYd5i2Y0= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= @@ -1319,8 +1299,6 @@ honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.2.0/go.mod h1:lPVVZ2BS5TfnjLyizF7o7hv7j9/L+8cZY2hLyjP9cGY= -k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/fileutil v1.0.0/go.mod h1:JHsWpkrk/CnVV1H/eGlFf85BEpfkrp56ro8nojIq9Q8= modernc.org/golex v1.0.1/go.mod h1:QCA53QtsT1NdGkaZZkF5ezFwk4IXh4BGNafAARTC254= modernc.org/lex v1.0.0/go.mod h1:G6rxMTy3cH2iA0iXL/HRRv4Znu8MK4higxph/lE7ypk= @@ -1335,13 +1313,12 @@ modernc.org/sortutil v1.0.0/go.mod h1:1QO0q8IlIlmjBIwm6t/7sof874+xCfZouyqZMLIAtx modernc.org/strutil v1.0.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= modernc.org/strutil v1.1.0/go.mod h1:lstksw84oURvj9y3tn8lGvRxyRC1S2+g5uuIzNfIOBs= modernc.org/y v1.0.1/go.mod h1:Ho86I+LVHEI+LYXoUKlmOMAM1JTXOCfj8qi1T8PsClE= -moul.io/zapgorm2 v1.1.0/go.mod h1:emRfKjNqSzVj5lcgasBdovIXY1jSOwFz2GQZn1Rddks= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= -sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= +sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= +sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 395db578cf9ad..258f35141c0f7 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -58,7 +58,7 @@ type clusterTablesSuite struct { startTime time.Time } -func TestClusterTables(t *testing.T) { +func TestForClusterServerInfo(t *testing.T) { // setup suite var clean func() s := new(clusterTablesSuite) @@ -70,195 +70,218 @@ func TestClusterTables(t *testing.T) { defer s.httpServer.Close() defer s.rpcserver.Stop() - // subtests - t.Run("ForClusterServerInfo", SubTestForClusterServerInfo(s)) - t.Run("DataLockWaits", SubTestTestDataLockWaits(s)) - t.Run("DataLockWaitsPrivilege", SubTestDataLockWaitsPrivilege(s)) - t.Run("SelectClusterTable", SubTestSelectClusterTable(s)) - t.Run("SelectClusterTablePrivilege", SubTestSelectClusterTablePrivilege(s)) - t.Run("StmtSummaryEvictedCountTable", SubTestStmtSummaryEvictedCountTable(s)) - t.Run("StmtSummaryHistoryTable", SubTestStmtSummaryHistoryTable(s)) - t.Run("Issue26379", SubTestIssue26379(s)) - t.Run("SubTestStmtSummaryResultRows", SubTestStmtSummaryResultRows(s)) -} + tk := testkit.NewTestKit(t, s.store) + instances := []string{ + strings.Join([]string{"tidb", s.listenAddr, s.listenAddr, "mock-version,mock-githash,1001"}, ","), + strings.Join([]string{"pd", s.listenAddr, s.listenAddr, "mock-version,mock-githash,0"}, ","), + strings.Join([]string{"tikv", s.listenAddr, s.listenAddr, "mock-version,mock-githash,0"}, ","), + } -func SubTestForClusterServerInfo(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := testkit.NewTestKit(t, s.store) - instances := []string{ - strings.Join([]string{"tidb", s.listenAddr, s.listenAddr, "mock-version,mock-githash,1001"}, ","), - strings.Join([]string{"pd", s.listenAddr, s.listenAddr, "mock-version,mock-githash,0"}, ","), - strings.Join([]string{"tikv", s.listenAddr, s.listenAddr, "mock-version,mock-githash,0"}, ","), - } + fpExpr := `return("` + strings.Join(instances, ";") + `")` + fpName := "github.com/pingcap/tidb/infoschema/mockClusterInfo" + require.NoError(t, failpoint.Enable(fpName, fpExpr)) + defer func() { require.NoError(t, failpoint.Disable(fpName)) }() + + cases := []struct { + sql string + types set.StringSet + addrs set.StringSet + names set.StringSet + skipOnDist set.StringSet + }{ + { + sql: "select * from information_schema.CLUSTER_LOAD;", + types: set.NewStringSet("tidb", "tikv", "pd"), + addrs: set.NewStringSet(s.listenAddr), + names: set.NewStringSet("cpu", "memory", "net"), + }, + { + sql: "select * from information_schema.CLUSTER_HARDWARE;", + types: set.NewStringSet("tidb", "tikv", "pd"), + addrs: set.NewStringSet(s.listenAddr), + names: set.NewStringSet("cpu", "memory", "net", "disk"), + // The sysutil package will filter out all disk don't have /dev prefix. + // gopsutil cpu.Info will fail on mac M1 + skipOnDist: set.NewStringSet("windows", "darwin/arm64"), + }, + { + sql: "select * from information_schema.CLUSTER_SYSTEMINFO;", + types: set.NewStringSet("tidb", "tikv", "pd"), + addrs: set.NewStringSet(s.listenAddr), + names: set.NewStringSet("system"), + // This test get empty result and fails on the windows platform. + // Because the underlying implementation use `sysctl` command to get the result + // and there is no such command on windows. + // https://github.com/pingcap/sysutil/blob/2bfa6dc40bcd4c103bf684fba528ae4279c7ec9f/system_info.go#L50 + skipOnDist: set.NewStringSet("windows"), + }, + } - fpExpr := `return("` + strings.Join(instances, ";") + `")` - fpName := "github.com/pingcap/tidb/infoschema/mockClusterInfo" - require.NoError(t, failpoint.Enable(fpName, fpExpr)) - defer func() { require.NoError(t, failpoint.Disable(fpName)) }() - - cases := []struct { - sql string - types set.StringSet - addrs set.StringSet - names set.StringSet - skipOnDist set.StringSet - }{ - { - sql: "select * from information_schema.CLUSTER_LOAD;", - types: set.NewStringSet("tidb", "tikv", "pd"), - addrs: set.NewStringSet(s.listenAddr), - names: set.NewStringSet("cpu", "memory", "net"), - }, - { - sql: "select * from information_schema.CLUSTER_HARDWARE;", - types: set.NewStringSet("tidb", "tikv", "pd"), - addrs: set.NewStringSet(s.listenAddr), - names: set.NewStringSet("cpu", "memory", "net", "disk"), - // The sysutil package will filter out all disk don't have /dev prefix. - // gopsutil cpu.Info will fail on mac M1 - skipOnDist: set.NewStringSet("windows", "darwin/arm64"), - }, - { - sql: "select * from information_schema.CLUSTER_SYSTEMINFO;", - types: set.NewStringSet("tidb", "tikv", "pd"), - addrs: set.NewStringSet(s.listenAddr), - names: set.NewStringSet("system"), - // This test get empty result and fails on the windows platform. - // Because the underlying implementation use `sysctl` command to get the result - // and there is no such command on windows. - // https://github.com/pingcap/sysutil/blob/2bfa6dc40bcd4c103bf684fba528ae4279c7ec9f/system_info.go#L50 - skipOnDist: set.NewStringSet("windows"), - }, + for _, cas := range cases { + if cas.skipOnDist.Exist(runtime.GOOS+"/"+runtime.GOARCH) || cas.skipOnDist.Exist(runtime.GOOS) { + continue } - for _, cas := range cases { - if cas.skipOnDist.Exist(runtime.GOOS+"/"+runtime.GOARCH) || cas.skipOnDist.Exist(runtime.GOOS) { - continue - } - - result := tk.MustQuery(cas.sql) - rows := result.Rows() - require.Greater(t, len(rows), 0) + result := tk.MustQuery(cas.sql) + rows := result.Rows() + require.Greater(t, len(rows), 0) - gotTypes := set.StringSet{} - gotAddrs := set.StringSet{} - gotNames := set.StringSet{} + gotTypes := set.StringSet{} + gotAddrs := set.StringSet{} + gotNames := set.StringSet{} - for _, row := range rows { - gotTypes.Insert(row[0].(string)) - gotAddrs.Insert(row[1].(string)) - gotNames.Insert(row[2].(string)) - } - - require.Equalf(t, cas.types, gotTypes, "sql: %s", cas.sql) - require.Equalf(t, cas.addrs, gotAddrs, "sql: %s", cas.sql) - require.Equalf(t, cas.names, gotNames, "sql: %s", cas.sql) + for _, row := range rows { + gotTypes.Insert(row[0].(string)) + gotAddrs.Insert(row[1].(string)) + gotNames.Insert(row[2].(string)) } + + require.Equalf(t, cas.types, gotTypes, "sql: %s", cas.sql) + require.Equalf(t, cas.addrs, gotAddrs, "sql: %s", cas.sql) + require.Equalf(t, cas.names, gotNames, "sql: %s", cas.sql) } } -func SubTestTestDataLockWaits(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - _, digest1 := parser.NormalizeDigest("select * from test_data_lock_waits for update") - _, digest2 := parser.NormalizeDigest("update test_data_lock_waits set f1=1 where id=2") - s.store.(mockstorage.MockLockWaitSetter).SetMockLockWaits([]*deadlock.WaitForEntry{ - {Txn: 1, WaitForTxn: 2, Key: []byte("key1"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest1, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, - {Txn: 3, WaitForTxn: 4, Key: []byte("key2"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest2, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, - // Invalid digests - {Txn: 5, WaitForTxn: 6, Key: []byte("key3"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(nil, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, - {Txn: 7, WaitForTxn: 8, Key: []byte("key4"), ResourceGroupTag: []byte("asdfghjkl")}, - }) - - tk := s.newTestKitWithRoot(t) - - // Execute one of the query once, so it's stored into statements_summary. - tk.MustExec("create table test_data_lock_waits (id int primary key, f1 int)") - tk.MustExec("select * from test_data_lock_waits for update") - - tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS").Check(testkit.Rows( - "6B657931 1 2 "+digest1.String()+" select * from `test_data_lock_waits` for update", - "6B657932 3 4 "+digest2.String()+" ", - "6B657933 5 6 ", - "6B657934 7 8 ")) - } +func TestTestDataLockWaits(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + + _, digest1 := parser.NormalizeDigest("select * from test_data_lock_waits for update") + _, digest2 := parser.NormalizeDigest("update test_data_lock_waits set f1=1 where id=2") + s.store.(mockstorage.MockLockWaitSetter).SetMockLockWaits([]*deadlock.WaitForEntry{ + {Txn: 1, WaitForTxn: 2, Key: []byte("key1"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest1, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, + {Txn: 3, WaitForTxn: 4, Key: []byte("key2"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest2, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, + // Invalid digests + {Txn: 5, WaitForTxn: 6, Key: []byte("key3"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(nil, nil, tipb.ResourceGroupTagLabel_ResourceGroupTagLabelUnknown)}, + {Txn: 7, WaitForTxn: 8, Key: []byte("key4"), ResourceGroupTag: []byte("asdfghjkl")}, + }) + + tk := s.newTestKitWithRoot(t) + + // Execute one of the query once, so it's stored into statements_summary. + tk.MustExec("create table test_data_lock_waits (id int primary key, f1 int)") + tk.MustExec("select * from test_data_lock_waits for update") + + tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS").Check(testkit.Rows( + "6B657931 1 2 "+digest1.String()+" select * from `test_data_lock_waits` for update", + "6B657932 3 4 "+digest2.String()+" ", + "6B657933 5 6 ", + "6B657934 7 8 ")) + } -func SubTestDataLockWaitsPrivilege(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - dropUserTk := s.newTestKitWithRoot(t) - - tk := s.newTestKitWithRoot(t) - - tk.MustExec("create user 'testuser'@'localhost'") - defer dropUserTk.MustExec("drop user 'testuser'@'localhost'") - require.True(t, tk.Session().Auth(&auth.UserIdentity{ - Username: "testuser", - Hostname: "localhost", - }, nil, nil)) - err := tk.QueryToErr("select * from information_schema.DATA_LOCK_WAITS") - require.EqualError(t, err, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") - - tk = s.newTestKitWithRoot(t) - tk.MustExec("create user 'testuser2'@'localhost'") - defer dropUserTk.MustExec("drop user 'testuser2'@'localhost'") - tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") - require.True(t, tk.Session().Auth(&auth.UserIdentity{ - Username: "testuser2", - Hostname: "localhost", - }, nil, nil)) - _ = tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS") - } +func SubTestDataLockWaitsPrivilege(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + + dropUserTk := s.newTestKitWithRoot(t) + + tk := s.newTestKitWithRoot(t) + + tk.MustExec("create user 'testuser'@'localhost'") + defer dropUserTk.MustExec("drop user 'testuser'@'localhost'") + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "testuser", + Hostname: "localhost", + }, nil, nil)) + err := tk.QueryToErr("select * from information_schema.DATA_LOCK_WAITS") + require.EqualError(t, err, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + tk = s.newTestKitWithRoot(t) + tk.MustExec("create user 'testuser2'@'localhost'") + defer dropUserTk.MustExec("drop user 'testuser2'@'localhost'") + tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "testuser2", + Hostname: "localhost", + }, nil, nil)) + _ = tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS") + } -func SubTestSelectClusterTable(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := s.newTestKitWithRoot(t) - slowLogFileName := "tidb-slow.log" - prepareSlowLogfile(t, slowLogFileName) - defer func() { require.NoError(t, os.Remove(slowLogFileName)) }() - for i := 0; i < 2; i++ { - tk.MustExec("use information_schema") - tk.MustExec(fmt.Sprintf("set @@tidb_enable_streaming=%d", i)) - tk.MustExec("set @@global.tidb_enable_stmt_summary=1") - tk.MustExec("set time_zone = '+08:00';") - tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("2")) - tk.MustQuery("select time from `CLUSTER_SLOW_QUERY` where time='2019-02-12 19:33:56.571953'").Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953")) - tk.MustQuery("select count(*) from `CLUSTER_PROCESSLIST`").Check(testkit.Rows("1")) - tk.MustQuery("select * from `CLUSTER_PROCESSLIST`").Check(testkit.Rows(fmt.Sprintf(":10080 1 root 127.0.0.1 Query 9223372036 %s 0 0 ", ""))) - tk.MustQuery("select query_time, conn_id from `CLUSTER_SLOW_QUERY` order by time limit 1").Check(testkit.Rows("4.895492 6")) - tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` group by digest").Check(testkit.Rows("1", "1")) - tk.MustQuery("select digest, count(*) from `CLUSTER_SLOW_QUERY` group by digest order by digest").Check(testkit.Rows("124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc 1", "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 1")) - tk.MustQuery(`select length(query) as l,time from information_schema.cluster_slow_query where time > "2019-02-12 19:33:56" order by abs(l) desc limit 10;`).Check(testkit.Rows("21 2019-02-12 19:33:56.571953")) - tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` where time > now() group by digest").Check(testkit.Rows()) - re := tk.MustQuery("select * from `CLUSTER_statements_summary`") - require.NotNil(t, re) - require.Greater(t, len(re.Rows()), 0) - // Test for TiDB issue 14915. - re = tk.MustQuery("select sum(exec_count*avg_mem) from cluster_statements_summary_history group by schema_name,digest,digest_text;") - require.NotNil(t, re) - require.Greater(t, len(re.Rows()), 0) - tk.MustQuery("select * from `CLUSTER_statements_summary_history`") - require.NotNil(t, re) - require.Greater(t, len(re.Rows()), 0) - tk.MustExec("set @@global.tidb_enable_stmt_summary=0") - re = tk.MustQuery("select * from `CLUSTER_statements_summary`") - require.NotNil(t, re) - require.Equal(t, 0, len(re.Rows())) - tk.MustQuery("select * from `CLUSTER_statements_summary_history`") - require.NotNil(t, re) - require.Equal(t, 0, len(re.Rows())) - } +func TestSelectClusterTable(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + slowLogFileName := "tidb-slow.log" + prepareSlowLogfile(t, slowLogFileName) + defer func() { require.NoError(t, os.Remove(slowLogFileName)) }() + for i := 0; i < 2; i++ { + tk.MustExec("use information_schema") + tk.MustExec(fmt.Sprintf("set @@tidb_enable_streaming=%d", i)) + tk.MustExec("set @@global.tidb_enable_stmt_summary=1") + tk.MustExec("set time_zone = '+08:00';") + tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("2")) + tk.MustQuery("select time from `CLUSTER_SLOW_QUERY` where time='2019-02-12 19:33:56.571953'").Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953")) + tk.MustQuery("select count(*) from `CLUSTER_PROCESSLIST`").Check(testkit.Rows("1")) + tk.MustQuery("select * from `CLUSTER_PROCESSLIST`").Check(testkit.Rows(fmt.Sprintf(":10080 1 root 127.0.0.1 Query 9223372036 %s 0 0 ", ""))) + tk.MustQuery("select query_time, conn_id from `CLUSTER_SLOW_QUERY` order by time limit 1").Check(testkit.Rows("4.895492 6")) + tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` group by digest").Check(testkit.Rows("1", "1")) + tk.MustQuery("select digest, count(*) from `CLUSTER_SLOW_QUERY` group by digest order by digest").Check(testkit.Rows("124acb3a0bec903176baca5f9da00b4e7512a41c93b417923f26502edeb324cc 1", "42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 1")) + tk.MustQuery(`select length(query) as l,time from information_schema.cluster_slow_query where time > "2019-02-12 19:33:56" order by abs(l) desc limit 10;`).Check(testkit.Rows("21 2019-02-12 19:33:56.571953")) + tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY` where time > now() group by digest").Check(testkit.Rows()) + re := tk.MustQuery("select * from `CLUSTER_statements_summary`") + require.NotNil(t, re) + require.Greater(t, len(re.Rows()), 0) + // Test for TiDB issue 14915. + re = tk.MustQuery("select sum(exec_count*avg_mem) from cluster_statements_summary_history group by schema_name,digest,digest_text;") + require.NotNil(t, re) + require.Greater(t, len(re.Rows()), 0) + tk.MustQuery("select * from `CLUSTER_statements_summary_history`") + require.NotNil(t, re) + require.Greater(t, len(re.Rows()), 0) + tk.MustExec("set @@global.tidb_enable_stmt_summary=0") + re = tk.MustQuery("select * from `CLUSTER_statements_summary`") + require.NotNil(t, re) + require.Equal(t, 0, len(re.Rows())) + tk.MustQuery("select * from `CLUSTER_statements_summary_history`") + require.NotNil(t, re) + require.Equal(t, 0, len(re.Rows())) } + } -func SubTestSelectClusterTablePrivilege(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := testkit.NewTestKit(t, s.store) - slowLogFileName := "tidb-slow.log" - f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) - require.NoError(t, err) - _, err = f.Write([]byte( - `# Time: 2019-02-12T19:33:57.571953+08:00 +func SubTestSelectClusterTablePrivilege(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := testkit.NewTestKit(t, s.store) + slowLogFileName := "tidb-slow.log" + f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) + require.NoError(t, err) + _, err = f.Write([]byte( + `# Time: 2019-02-12T19:33:57.571953+08:00 # User@Host: user2 [user2] @ 127.0.0.1 [127.0.0.1] select * from t2; # Time: 2019-02-12T19:33:56.571953+08:00 @@ -270,225 +293,312 @@ select * from t3; # Time: 2019-02-12T19:33:59.571953+08:00 select * from t3; `)) - require.NoError(t, f.Close()) - require.NoError(t, err) - defer func() { require.NoError(t, os.Remove(slowLogFileName)) }() - tk.MustExec("use information_schema") - tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("4")) - tk.MustQuery("select count(*) from `SLOW_QUERY`").Check(testkit.Rows("4")) - tk.MustQuery("select count(*) from `CLUSTER_PROCESSLIST`").Check(testkit.Rows("1")) - tk.MustQuery("select * from `CLUSTER_PROCESSLIST`").Check(testkit.Rows(fmt.Sprintf(":10080 1 root 127.0.0.1 Query 9223372036 %s 0 0 ", ""))) - tk.MustExec("create user user1") - tk.MustExec("create user user2") - user1 := testkit.NewTestKit(t, s.store) - user1.MustExec("use information_schema") - require.True(t, user1.Session().Auth(&auth.UserIdentity{ - Username: "user1", - Hostname: "127.0.0.1", - }, nil, nil)) - user1.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("1")) - user1.MustQuery("select count(*) from `SLOW_QUERY`").Check(testkit.Rows("1")) - user1.MustQuery("select user,query from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("user1 select * from t1;")) - - user2 := testkit.NewTestKit(t, s.store) - user2.MustExec("use information_schema") - require.True(t, user2.Session().Auth(&auth.UserIdentity{ - Username: "user2", - Hostname: "127.0.0.1", - }, nil, nil)) - user2.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("2")) - user2.MustQuery("select user,query from `CLUSTER_SLOW_QUERY` order by query").Check(testkit.Rows("user2 select * from t2;", "user2 select * from t3;")) - } + require.NoError(t, f.Close()) + require.NoError(t, err) + defer func() { require.NoError(t, os.Remove(slowLogFileName)) }() + tk.MustExec("use information_schema") + tk.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from `SLOW_QUERY`").Check(testkit.Rows("4")) + tk.MustQuery("select count(*) from `CLUSTER_PROCESSLIST`").Check(testkit.Rows("1")) + tk.MustQuery("select * from `CLUSTER_PROCESSLIST`").Check(testkit.Rows(fmt.Sprintf(":10080 1 root 127.0.0.1 Query 9223372036 %s 0 0 ", ""))) + tk.MustExec("create user user1") + tk.MustExec("create user user2") + user1 := testkit.NewTestKit(t, s.store) + user1.MustExec("use information_schema") + require.True(t, user1.Session().Auth(&auth.UserIdentity{ + Username: "user1", + Hostname: "127.0.0.1", + }, nil, nil)) + user1.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("1")) + user1.MustQuery("select count(*) from `SLOW_QUERY`").Check(testkit.Rows("1")) + user1.MustQuery("select user,query from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("user1 select * from t1;")) + + user2 := testkit.NewTestKit(t, s.store) + user2.MustExec("use information_schema") + require.True(t, user2.Session().Auth(&auth.UserIdentity{ + Username: "user2", + Hostname: "127.0.0.1", + }, nil, nil)) + user2.MustQuery("select count(*) from `CLUSTER_SLOW_QUERY`").Check(testkit.Rows("2")) + user2.MustQuery("select user,query from `CLUSTER_SLOW_QUERY` order by query").Check(testkit.Rows("user2 select * from t2;", "user2 select * from t3;")) } -func SubTestStmtSummaryEvictedCountTable(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := s.newTestKitWithRoot(t) - // disable refreshing - tk.MustExec("set global tidb_stmt_summary_refresh_interval=9999") - // set information_schema.statements_summary's size to 2 - tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 2") - // no evict happened, no record in cluster evicted table. - tk.MustQuery("select count(*) from information_schema.cluster_statements_summary_evicted;").Check(testkit.Rows("0")) - tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 1") - // cleanup side effects - defer tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 100") - defer tk.MustExec("set global tidb_stmt_summary_refresh_interval = 1800") - // clear information_schema.statements_summary - tk.MustExec("set global tidb_enable_stmt_summary=0") - // statements_summary is off, statements_summary_evicted is empty. - tk.MustQuery("select count(*) from information_schema.cluster_statements_summary_evicted;").Check(testkit.Rows("0")) - tk.MustExec("set global tidb_enable_stmt_summary=1") - - // make a new session for test... - tk = s.newTestKitWithRoot(t) - // first sql - tk.MustExec("show databases;") - // second sql, evict former sql from stmt_summary - tk.MustQuery("select evicted_count from information_schema.cluster_statements_summary_evicted;"). - Check(testkit.Rows("1")) - // after executed the sql above - tk.MustQuery("select evicted_count from information_schema.cluster_statements_summary_evicted;"). - Check(testkit.Rows("2")) - // TODO: Add more tests. - - tk.MustExec("create user 'testuser'@'localhost'") - tk.MustExec("create user 'testuser2'@'localhost'") - tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") - tk1 := s.newTestKitWithRoot(t) - defer tk1.MustExec("drop user 'testuser'@'localhost'") - defer tk1.MustExec("drop user 'testuser2'@'localhost'") - - require.True(t, tk.Session().Auth(&auth.UserIdentity{ - Username: "testuser", - Hostname: "localhost", - }, nil, nil)) - - err := tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED") - // This error is come from cop(TiDB) fetch from rpc server. - require.EqualError(t, err, "other error: [planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") - - require.True(t, tk.Session().Auth(&auth.UserIdentity{ - Username: "testuser2", - Hostname: "localhost", - }, nil, nil)) - require.NoError(t, tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED")) - } +func TestStmtSummaryEvictedCountTable(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + + tk := s.newTestKitWithRoot(t) + // disable refreshing + tk.MustExec("set global tidb_stmt_summary_refresh_interval=9999") + // set information_schema.statements_summary's size to 2 + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 2") + // no evict happened, no record in cluster evicted table. + tk.MustQuery("select count(*) from information_schema.cluster_statements_summary_evicted;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 1") + // cleanup side effects + defer tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 100") + defer tk.MustExec("set global tidb_stmt_summary_refresh_interval = 1800") + // clear information_schema.statements_summary + tk.MustExec("set global tidb_enable_stmt_summary=0") + // statements_summary is off, statements_summary_evicted is empty. + tk.MustQuery("select count(*) from information_schema.cluster_statements_summary_evicted;").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_enable_stmt_summary=1") + + // make a new session for test... + tk = s.newTestKitWithRoot(t) + // first sql + tk.MustExec("show databases;") + // second sql, evict former sql from stmt_summary + tk.MustQuery("select evicted_count from information_schema.cluster_statements_summary_evicted;"). + Check(testkit.Rows("1")) + // after executed the sql above + tk.MustQuery("select evicted_count from information_schema.cluster_statements_summary_evicted;"). + Check(testkit.Rows("2")) + // TODO: Add more tests. + + tk.MustExec("create user 'testuser'@'localhost'") + tk.MustExec("create user 'testuser2'@'localhost'") + tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") + tk1 := s.newTestKitWithRoot(t) + defer tk1.MustExec("drop user 'testuser'@'localhost'") + defer tk1.MustExec("drop user 'testuser2'@'localhost'") + + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "testuser", + Hostname: "localhost", + }, nil, nil)) + + err := tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED") + // This error is come from cop(TiDB) fetch from rpc server. + require.EqualError(t, err, "other error: [planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "testuser2", + Hostname: "localhost", + }, nil, nil)) + require.NoError(t, tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED")) +} + +func TestStmtSummaryHistoryTableWithUserTimezone(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + + tk := s.newTestKitWithRoot(t) + tk.MustExec("drop table if exists test_summary") + tk.MustExec("create table test_summary(a int, b varchar(10), key k(a))") + + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + + // Create a new session to test. + tk = s.newTestKitWithRoot(t) + tk.MustExec("use test;") + tk.MustExec("set time_zone = '+08:00';") + tk.MustExec("select sleep(0.1);") + r := tk.MustQuery("select FIRST_SEEN, LAST_SEEN, SUMMARY_BEGIN_TIME, SUMMARY_END_TIME from INFORMATION_SCHEMA.STATEMENTS_SUMMARY_HISTORY order by LAST_SEEN limit 1;") + date8First, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][0].(string)) + require.NoError(t, err) + date8Last, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][1].(string)) + require.NoError(t, err) + date8Begin, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][2].(string)) + require.NoError(t, err) + date8End, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][3].(string)) + require.NoError(t, err) + tk.MustExec("set time_zone = '+01:00';") + r = tk.MustQuery("select FIRST_SEEN, LAST_SEEN, SUMMARY_BEGIN_TIME, SUMMARY_END_TIME from INFORMATION_SCHEMA.STATEMENTS_SUMMARY_HISTORY order by LAST_SEEN limit 1;") + date1First, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][0].(string)) + require.NoError(t, err) + date1Last, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][1].(string)) + require.NoError(t, err) + date1Begin, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][2].(string)) + require.NoError(t, err) + date1End, err := time.Parse("2006-01-02 15:04:05", r.Rows()[0][3].(string)) + require.NoError(t, err) + + require.Less(t, date1First.Unix(), date8First.Unix()) + require.Less(t, date1Last.Unix(), date8Last.Unix()) + require.Less(t, date1Begin.Unix(), date8Begin.Unix()) + require.Less(t, date1End.Unix(), date8End.Unix()) } -func SubTestStmtSummaryHistoryTable(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := s.newTestKitWithRoot(t) - tk.MustExec("drop table if exists test_summary") - tk.MustExec("create table test_summary(a int, b varchar(10), key k(a))") - - tk.MustExec("set global tidb_enable_stmt_summary = 1") - tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - - // Disable refreshing summary. - tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") - tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) - - // Create a new session to test. - tk = s.newTestKitWithRoot(t) - - // Test INSERT - tk.MustExec("insert into test_summary values(1, 'a')") - tk.MustExec("insert into test_summary values(2, 'b')") - tk.MustExec("insert into TEST_SUMMARY VALUES(3, 'c')") - tk.MustExec("/**/insert into test_summary values(4, 'd')") - - sql := "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys," + - "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions," + - "max_prewrite_regions, avg_affected_rows, query_sample_text " + - "from information_schema.statements_summary_history " + - "where digest_text like 'insert into `test_summary`%'" - tk.MustQuery(sql).Check(testkit.Rows("Insert test test.test_summary 4 0 0 0 0 0 2 2 1 1 1 insert into test_summary values(1, 'a')")) - - tk.MustExec("set global tidb_stmt_summary_history_size = 0") - tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, +func TestStmtSummaryHistoryTable(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + + tk := s.newTestKitWithRoot(t) + tk.MustExec("drop table if exists test_summary") + tk.MustExec("create table test_summary(a int, b varchar(10), key k(a))") + + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) + + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + + // Create a new session to test. + tk = s.newTestKitWithRoot(t) + + // Test INSERT + tk.MustExec("insert into test_summary values(1, 'a')") + tk.MustExec("insert into test_summary values(2, 'b')") + tk.MustExec("insert into TEST_SUMMARY VALUES(3, 'c')") + tk.MustExec("/**/insert into test_summary values(4, 'd')") + + sql := "select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys," + + "max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions," + + "max_prewrite_regions, avg_affected_rows, query_sample_text " + + "from information_schema.statements_summary_history " + + "where digest_text like 'insert into `test_summary`%'" + tk.MustQuery(sql).Check(testkit.Rows("Insert test test.test_summary 4 0 0 0 0 0 2 2 1 1 1 insert into test_summary values(1, 'a')")) + + tk.MustExec("set global tidb_stmt_summary_history_size = 0") + tk.MustQuery(`select stmt_type, schema_name, table_names, index_names, exec_count, sum_cop_task_num, avg_total_keys, max_total_keys, avg_processed_keys, max_processed_keys, avg_write_keys, max_write_keys, avg_prewrite_regions, max_prewrite_regions, avg_affected_rows, query_sample_text, plan from information_schema.statements_summary_history`, - ).Check(testkit.Rows()) - - tk.MustExec("set global tidb_enable_stmt_summary = 0") - tk.MustExec("drop table if exists `table`") - tk.MustExec("set global tidb_stmt_summary_history_size = 1") - tk.MustExec("set global tidb_enable_stmt_summary = 1") - tk.MustExec("create table `table`(`insert` int)") - tk.MustExec("select `insert` from `table`") - - sql = "select digest_text from information_schema.statements_summary_history;" - tk.MustQuery(sql).Check(testkit.Rows( - "select `insert` from `table`", - "create table `table` ( `insert` int )", - "set global `tidb_enable_stmt_summary` = ?", - )) - } + ).Check(testkit.Rows()) + + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("drop table if exists `table`") + tk.MustExec("set global tidb_stmt_summary_history_size = 1") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustExec("create table `table`(`insert` int)") + tk.MustExec("select `insert` from `table`") + + sql = "select digest_text from information_schema.statements_summary_history;" + tk.MustQuery(sql).Check(testkit.Rows( + "select `insert` from `table`", + "create table `table` ( `insert` int )", + "set global `tidb_enable_stmt_summary` = ?", + )) } -func SubTestIssue26379(s *clusterTablesSuite) func(*testing.T) { - return func(t *testing.T) { - tk := s.newTestKitWithRoot(t) - - // Clear all statements. - tk.MustExec("set global tidb_enable_stmt_summary = 0") - tk.MustExec("set global tidb_enable_stmt_summary = 1") - tk.MustExec("set @@global.tidb_stmt_summary_max_stmt_count=10") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b varchar(10), c int, d int, key k(a))") - - _, digest1 := parser.NormalizeDigest("select * from t where a = 3") - _, digest2 := parser.NormalizeDigest("select * from t where b = 'b'") - _, digest3 := parser.NormalizeDigest("select * from t where c = 6") - _, digest4 := parser.NormalizeDigest("select * from t where d = 5") - fillStatementCache := func() { - tk.MustQuery("select * from t where a = 3") - tk.MustQuery("select * from t where b = 'b'") - tk.MustQuery("select * from t where c = 6") - tk.MustQuery("select * from t where d = 5") - } - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest1.String())).Check(testkit.Rows(digest1.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest1.String())).Check(testkit.Rows(digest1.String())) - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest2.String())).Check(testkit.Rows(digest2.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest2.String())).Check(testkit.Rows(digest2.String())) - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest3.String())).Check(testkit.Rows(digest3.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest3.String())).Check(testkit.Rows(digest3.String())) - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest4.String())).Check(testkit.Rows(digest4.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest4.String())).Check(testkit.Rows(digest4.String())) - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s' or digest = '%s'", digest1.String(), digest2.String())).Sort().Check(testkit.Rows(digest1.String(), digest2.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' or digest = '%s'", digest1.String(), digest2.String())).Sort().Check(testkit.Rows(digest1.String(), digest2.String())) - re := tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' and digest = '%s'", digest1.String(), digest2.String())) - require.Equal(t, 0, len(re.Rows())) - re = tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' and digest = '%s'", digest1.String(), digest2.String())) - require.Equal(t, 0, len(re.Rows())) - fillStatementCache() - tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest in ('%s', '%s', '%s', '%s')", digest1.String(), digest2.String(), digest3.String(), digest4.String())).Sort().Check(testkit.Rows(digest1.String(), digest4.String(), digest2.String(), digest3.String())) - tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest in ('%s', '%s', '%s', '%s')", digest1.String(), digest2.String(), digest3.String(), digest4.String())).Sort().Check(testkit.Rows(digest1.String(), digest4.String(), digest2.String(), digest3.String())) - fillStatementCache() - tk.MustQuery("select count(*) from information_schema.statements_summary where digest=''").Check(testkit.Rows("0")) - tk.MustQuery("select count(*) from information_schema.statements_summary where digest is null").Check(testkit.Rows("1")) - tk.MustQuery("select count(*) from information_schema.cluster_statements_summary where digest=''").Check(testkit.Rows("0")) - tk.MustQuery("select count(*) from information_schema.cluster_statements_summary where digest is null").Check(testkit.Rows("1")) +func TestIssue26379(t *testing.T) { + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + + // Clear all statements. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + tk.MustExec("set @@global.tidb_stmt_summary_max_stmt_count=10") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), c int, d int, key k(a))") + + _, digest1 := parser.NormalizeDigest("select * from t where a = 3") + _, digest2 := parser.NormalizeDigest("select * from t where b = 'b'") + _, digest3 := parser.NormalizeDigest("select * from t where c = 6") + _, digest4 := parser.NormalizeDigest("select * from t where d = 5") + fillStatementCache := func() { + tk.MustQuery("select * from t where a = 3") + tk.MustQuery("select * from t where b = 'b'") + tk.MustQuery("select * from t where c = 6") + tk.MustQuery("select * from t where d = 5") } + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest1.String())).Check(testkit.Rows(digest1.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest1.String())).Check(testkit.Rows(digest1.String())) + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest2.String())).Check(testkit.Rows(digest2.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest2.String())).Check(testkit.Rows(digest2.String())) + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest3.String())).Check(testkit.Rows(digest3.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest3.String())).Check(testkit.Rows(digest3.String())) + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s'", digest4.String())).Check(testkit.Rows(digest4.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s'", digest4.String())).Check(testkit.Rows(digest4.String())) + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest = '%s' or digest = '%s'", digest1.String(), digest2.String())).Sort().Check(testkit.Rows(digest1.String(), digest2.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' or digest = '%s'", digest1.String(), digest2.String())).Sort().Check(testkit.Rows(digest1.String(), digest2.String())) + re := tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' and digest = '%s'", digest1.String(), digest2.String())) + require.Equal(t, 0, len(re.Rows())) + re = tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest = '%s' and digest = '%s'", digest1.String(), digest2.String())) + require.Equal(t, 0, len(re.Rows())) + fillStatementCache() + tk.MustQuery(fmt.Sprintf("select digest from information_schema.statements_summary where digest in ('%s', '%s', '%s', '%s')", digest1.String(), digest2.String(), digest3.String(), digest4.String())).Sort().Check(testkit.Rows(digest1.String(), digest4.String(), digest2.String(), digest3.String())) + tk.MustQuery(fmt.Sprintf("select digest from information_schema.cluster_statements_summary where digest in ('%s', '%s', '%s', '%s')", digest1.String(), digest2.String(), digest3.String(), digest4.String())).Sort().Check(testkit.Rows(digest1.String(), digest4.String(), digest2.String(), digest3.String())) + fillStatementCache() + tk.MustQuery("select count(*) from information_schema.statements_summary where digest=''").Check(testkit.Rows("0")) + tk.MustQuery("select count(*) from information_schema.statements_summary where digest is null").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from information_schema.cluster_statements_summary where digest=''").Check(testkit.Rows("0")) + tk.MustQuery("select count(*) from information_schema.cluster_statements_summary where digest is null").Check(testkit.Rows("1")) } -func SubTestStmtSummaryResultRows(s *clusterTablesSuite) func(t *testing.T) { - return func(t *testing.T) { - tk := s.newTestKitWithRoot(t) - tk.MustExec("set global tidb_stmt_summary_refresh_interval=999999999") - tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 3000") - tk.MustExec("set global tidb_stmt_summary_history_size=24") - tk.MustExec("set global tidb_stmt_summary_max_sql_length=4096") - tk.MustExec("set global tidb_enable_stmt_summary=0") - tk.MustExec("set global tidb_enable_stmt_summary=1") - if !config.GetGlobalConfig().EnableCollectExecutionInfo { - tk.MustExec("set @@tidb_enable_collect_execution_info=1") - defer tk.MustExec("set @@tidb_enable_collect_execution_info=0") - } +func TestStmtSummaryResultRows(t *testing.T) { + // setup suite + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0") + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a int)") - for i := 1; i <= 30; i++ { - tk.MustExec(fmt.Sprintf("insert into t values (%v)", i)) - } + tk := s.newTestKitWithRoot(t) + tk.MustExec("set global tidb_stmt_summary_refresh_interval=999999999") + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 3000") + tk.MustExec("set global tidb_stmt_summary_history_size=24") + tk.MustExec("set global tidb_stmt_summary_max_sql_length=4096") + tk.MustExec("set global tidb_enable_stmt_summary=0") + tk.MustExec("set global tidb_enable_stmt_summary=1") + if !config.GetGlobalConfig().EnableCollectExecutionInfo { + tk.MustExec("set @@tidb_enable_collect_execution_info=1") + defer tk.MustExec("set @@tidb_enable_collect_execution_info=0") + } - tk.MustQuery("select * from test.t limit 10;") - tk.MustQuery("select * from test.t limit 20;") - tk.MustQuery("select * from test.t limit 30;") - tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10"). - Check(testkit.Rows("10 30 20")) - tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.cluster_statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10"). - Check(testkit.Rows("10 30 20")) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int)") + for i := 1; i <= 30; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%v)", i)) } + + tk.MustQuery("select * from test.t limit 10;") + tk.MustQuery("select * from test.t limit 20;") + tk.MustQuery("select * from test.t limit 30;") + tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10"). + Check(testkit.Rows("10 30 20")) + tk.MustQuery("select MIN_RESULT_ROWS,MAX_RESULT_ROWS,AVG_RESULT_ROWS from information_schema.cluster_statements_summary where query_sample_text like 'select%test.t limit%' and MAX_RESULT_ROWS > 10"). + Check(testkit.Rows("10 30 20")) } func (s *clusterTablesSuite) setUpRPCService(t *testing.T, addr string) (*grpc.Server, string) { diff --git a/infoschema/error.go b/infoschema/error.go index 49dc2136d0820..6afcf23177e02 100644 --- a/infoschema/error.go +++ b/infoschema/error.go @@ -78,4 +78,6 @@ var ( ErrWrongObject = dbterror.ClassSchema.NewStd(mysql.ErrWrongObject) // ErrAdminCheckTable returns when the check table in temporary mode. ErrAdminCheckTable = dbterror.ClassSchema.NewStd(mysql.ErrAdminCheckTable) + // ErrEmptyDatabase returns when the database is unexpectedly empty. + ErrEmptyDatabase = dbterror.ClassSchema.NewStd(mysql.ErrBadDB) ) diff --git a/infoschema/main_test.go b/infoschema/main_test.go index 8a878a925be01..a7a4c26e3161a 100644 --- a/infoschema/main_test.go +++ b/infoschema/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/infoschema/perfschema/main_test.go b/infoschema/perfschema/main_test.go index a13c7d1ddca79..ff4b9ed022c02 100644 --- a/infoschema/perfschema/main_test.go +++ b/infoschema/perfschema/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/infoschema/tables.go b/infoschema/tables.go index afee75ce12490..84b1296e3b517 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1160,6 +1160,7 @@ var tableDDLJobsCols = []columnInfo{ {name: "SCHEMA_ID", tp: mysql.TypeLonglong, size: 21}, {name: "TABLE_ID", tp: mysql.TypeLonglong, size: 21}, {name: "ROW_COUNT", tp: mysql.TypeLonglong, size: 21}, + {name: "CREATE_TIME", tp: mysql.TypeDatetime, size: 19}, {name: "START_TIME", tp: mysql.TypeDatetime, size: 19}, {name: "END_TIME", tp: mysql.TypeDatetime, size: 19}, {name: "STATE", tp: mysql.TypeVarchar, size: 64}, diff --git a/kv/error.go b/kv/error.go index f6ad87eb4cdf1..9a6878d57cebd 100644 --- a/kv/error.go +++ b/kv/error.go @@ -54,6 +54,8 @@ var ( pmysql.Message(mysql.MySQLErrName[mysql.ErrWriteConflictInTiDB].Raw+" "+TxnRetryableMark, nil)) // ErrLockExpire is the error when the lock is expired. ErrLockExpire = dbterror.ClassTiKV.NewStd(mysql.ErrLockExpire) + // ErrAssertionFailed is the error when an assertion fails. + ErrAssertionFailed = dbterror.ClassTiKV.NewStd(mysql.ErrAssertionFailed) ) // IsTxnRetryableError checks if the error could safely retry the transaction. diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index c8743284c9a8c..3412b5b717d92 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -33,10 +33,10 @@ func TestFaultInjectionBasic(t *testing.T) { storage := NewInjectedStore(newMockStorage(), &cfg) txn, err := storage.Begin() - require.Nil(t, err) + require.NoError(t, err) _, err = storage.Begin(tikv.WithTxnScope(GlobalTxnScope), tikv.WithStartTS(0)) - require.Nil(t, err) + require.NoError(t, err) ver := Version{Ver: 1} snap := storage.GetSnapshot(ver) diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 2542e6f7e5b55..26f9acf1d6cfb 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -30,6 +30,10 @@ type mockTxn struct { valid bool } +func (t *mockTxn) SetAssertion(_ []byte, _ ...FlagsOp) error { + return nil +} + // Commit always returns a retryable error. func (t *mockTxn) Commit(ctx context.Context) error { return ErrTxnRetryable diff --git a/kv/key_test.go b/kv/key_test.go index 96b4fd2d8dfb4..04f7e2979e07d 100644 --- a/kv/key_test.go +++ b/kv/key_test.go @@ -33,13 +33,13 @@ func TestPartialNext(t *testing.T) { sc := &stmtctx.StatementContext{TimeZone: time.Local} // keyA represents a multi column index. keyA, err := codec.EncodeValue(sc, nil, types.NewDatum("abc"), types.NewDatum("def")) - require.Nil(t, err) + require.NoError(t, err) keyB, err := codec.EncodeValue(sc, nil, types.NewDatum("abca"), types.NewDatum("def")) - require.Nil(t, err) + require.NoError(t, err) // We only use first column value to seek. seekKey, err := codec.EncodeValue(sc, nil, types.NewDatum("abc")) - require.Nil(t, err) + require.NoError(t, err) nextKey := Key(seekKey).Next() cmp := bytes.Compare(nextKey, keyA) diff --git a/kv/keyflags.go b/kv/keyflags.go index 66f94380c7a50..27f8472cf4779 100644 --- a/kv/keyflags.go +++ b/kv/keyflags.go @@ -20,6 +20,17 @@ type KeyFlags uint8 const ( flagPresumeKNE KeyFlags = 1 << iota flagNeedLocked + + // The following are assertion related flags. + // There are four choices of the two bits: + // * 0: Assertion is not set and can be set later. + // * flagAssertExists: We assert the key exists. + // * flagAssertNotExists: We assert the key doesn't exist. + // * flagAssertExists | flagAssertNotExists: Assertion cannot be made on this key (unknown). + // Once either (or both) of the two flags is set, we say assertion is set (`HasAssertionFlags` becomes true), and + // it's expected to be unchangeable within the current transaction. + flagAssertExists + flagAssertNotExists ) // HasPresumeKeyNotExists returns whether the associated key use lazy check. @@ -32,6 +43,26 @@ func (f KeyFlags) HasNeedLocked() bool { return f&flagNeedLocked != 0 } +// HasAssertExists returns whether the key is asserted to already exist before the current transaction. +func (f KeyFlags) HasAssertExists() bool { + return f&flagAssertExists != 0 && f&flagAssertNotExists == 0 +} + +// HasAssertNotExists returns whether the key is asserted not to exist before the current transaction. +func (f KeyFlags) HasAssertNotExists() bool { + return f&flagAssertNotExists != 0 && f&flagAssertExists == 0 +} + +// HasAssertUnknown returns whether the key is unable to do any assertion. +func (f KeyFlags) HasAssertUnknown() bool { + return f&flagAssertExists != 0 && f&flagAssertNotExists != 0 +} + +// HasAssertionFlags returns whether assertion is set on this key. +func (f KeyFlags) HasAssertionFlags() bool { + return f&flagAssertExists != 0 || f&flagAssertNotExists != 0 +} + // FlagsOp describes KeyFlags modify operation. type FlagsOp uint16 @@ -40,6 +71,14 @@ const ( SetPresumeKeyNotExists FlagsOp = iota // SetNeedLocked marks the associated key need to be acquired lock. SetNeedLocked + // SetAssertExist marks the associated key must exist. + SetAssertExist + // SetAssertNotExist marks the associated key must not exists. + SetAssertNotExist + // SetAssertUnknown marks the associated key is unknown and can not apply other assertion. + SetAssertUnknown + // SetAssertNone marks the associated key without any assert. + SetAssertNone ) // ApplyFlagsOps applys flagspos to origin. @@ -50,6 +89,15 @@ func ApplyFlagsOps(origin KeyFlags, ops ...FlagsOp) KeyFlags { origin |= flagPresumeKNE case SetNeedLocked: origin |= flagNeedLocked + case SetAssertExist: + origin |= flagAssertExists + origin &= ^flagAssertNotExists + case SetAssertNotExist: + origin |= flagAssertNotExists + origin &= ^flagAssertExists + case SetAssertUnknown: + origin |= flagAssertExists + origin |= flagAssertNotExists } } return origin diff --git a/kv/kv.go b/kv/kv.go index 742fa7a793c08..df6d985fee59a 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -31,6 +31,7 @@ import ( "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" + pd "github.com/tikv/pd/client" ) // UnCommitIndexKVFlag uses to indicate the index key/value is no need to commit. @@ -182,6 +183,7 @@ type LockCtx = tikvstore.LockCtx // This is not thread safe. type Transaction interface { RetrieverMutator + AssertionProto // Size returns sum of keys and values length. Size() int // Len returns the number of entries in the DB. @@ -235,6 +237,14 @@ type Transaction interface { ClearDiskFullOpt() } +// AssertionProto is an interface defined for the assertion protocol. +type AssertionProto interface { + // SetAssertion sets an assertion for an operation on the key. + // TODO: Use a special type instead of `FlagsOp`. Otherwise there's risk that the assertion flag is incorrectly used + // in other places like `MemBuffer.SetWithFlags`. + SetAssertion(key []byte, assertion ...FlagsOp) error +} + // Client is used to send request to KV layer. type Client interface { // Send sends request to KV layer, returns a Response. @@ -455,6 +465,11 @@ type EtcdBackend interface { StartGCWorker() error } +// StorageWithPD is used to get pd client. +type StorageWithPD interface { + GetPDClient() pd.Client +} + // FnKeyCmp is the function for iterator the keys type FnKeyCmp func(key Key) bool diff --git a/kv/main_test.go b/kv/main_test.go index 6d1f289de7d9a..40587cb4acdff 100644 --- a/kv/main_test.go +++ b/kv/main_test.go @@ -25,7 +25,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/kv/option.go b/kv/option.go index f3247b5b31b2c..2e9418e755e3e 100644 --- a/kv/option.go +++ b/kv/option.go @@ -76,6 +76,11 @@ const ( // RPCInterceptor is interceptor.RPCInterceptor on Transaction or Snapshot, used to decorate // additional logic before and after the underlying client-go RPC request. RPCInterceptor + // TableToColumnMaps is a map from tableID to a series of maps. The maps are needed when checking data consistency. + // Save them here to reduce redundant computations. + TableToColumnMaps + // AssertionLevel controls how strict the assertions on data during transactions should be. + AssertionLevel ) // ReplicaReadType is the type of replica to read data from diff --git a/meta/main_test.go b/meta/main_test.go index 7fd569a01a417..59fbb4a95b08c 100644 --- a/meta/main_test.go +++ b/meta/main_test.go @@ -25,7 +25,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), } goleak.VerifyTestMain(m, opts...) } diff --git a/meta/meta.go b/meta/meta.go index a48565ecb9b9f..9a92a9380848d 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -918,6 +918,11 @@ func (m *Meta) GetAllHistoryDDLJobs() ([]*model.Job, error) { return jobs, nil } +// GetHistoryDDLCount the count of all history DDL jobs. +func (m *Meta) GetHistoryDDLCount() (uint64, error) { + return m.txn.HGetLen(mDDLJobHistoryKey) +} + // GetLastNHistoryDDLJobs gets latest N history ddl jobs. func (m *Meta) GetLastNHistoryDDLJobs(num int) ([]*model.Job, error) { pairs, err := m.txn.HGetLastN(mDDLJobHistoryKey, num) diff --git a/metrics/grafana/README.md b/metrics/grafana/README.md new file mode 100644 index 0000000000000..8b5faaaf1649c --- /dev/null +++ b/metrics/grafana/README.md @@ -0,0 +1,14 @@ +## About + +Use [jsonnet](https://github.com/google/go-jsonnet) to generate Grafana-compliant json scripts for use with TiDB. + +Why jsonnet? + +1. jsonnet is a DSL created by Google for json, which is good for advanced json editing work. +2. Grafana provides the [jsonnet library](https://grafana.github.io/grafonnet-lib/) specifically for generating Grafana json, which makes maintaining TiDB's json scripts much easier. + +## Usage + +1. Modify the jsonnet files (e.g. tidb_summary.jsonnet). +2. Run `generate_json.sh` to generate the json files by the jsonnet files. +3. Commit the modifications. \ No newline at end of file diff --git a/metrics/grafana/generate_json.sh b/metrics/grafana/generate_json.sh new file mode 100755 index 0000000000000..7739382813610 --- /dev/null +++ b/metrics/grafana/generate_json.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +go install github.com/google/go-jsonnet/cmd/jsonnet@latest +git clone https://github.com/grafana/grafonnet-lib.git + +export JSONNET_PATH=grafonnet-lib +jsonnet tidb_summary.jsonnet > tidb_summary.json +rm -rf $JSONNET_PATH + diff --git a/metrics/grafana/performance_overview.json b/metrics/grafana/performance_overview.json index dadce37b3c847..e1873f6a39b85 100644 --- a/metrics/grafana/performance_overview.json +++ b/metrics/grafana/performance_overview.json @@ -79,6 +79,157 @@ "title": "Performance Overview", "type": "row" }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Service Time Per Second, show service time distribution among different components and execution phase.\n1. Total Database Time, the time which tidb cluster is processing application requests.\n2. Per SQL Type dimension\n3. Per parse/compile/execute dimension\n4. Per KV/PD request dimension", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 1 + }, + "hiddenSeries": false, + "id": 187, + "legend": { + "alignAsTable": true, + "avg": true, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sort": "avg", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "database time", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_request_seconds_sum{tidb_cluster=\"$tidb_cluster\", store!=\"0\"}[1m])) by (type)", + "hide": false, + "interval": "", + "legendFormat": "kv_request - {{type}}", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "tso_wait", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_session_parse_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "sql - parse", + "refId": "D" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_session_compile_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "sql - compile", + "refId": "E" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_session_execute_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "sql - execute", + "refId": "F" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (sql_type)", + "hide": false, + "interval": "", + "legendFormat": "sql_type - {{sql_type}}", + "refId": "G" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Database Time Overview", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "aliasColors": {}, "bars": false, @@ -99,7 +250,7 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, + "x": 12, "y": 1 }, "hiddenSeries": false, @@ -218,8 +369,8 @@ "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 1 + "x": 0, + "y": 8 }, "hiddenSeries": false, "id": 178, @@ -316,7 +467,7 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "kv request total by instance and command type", + "description": "TiDB plan cache hit total.", "editable": true, "error": false, "fieldConfig": { @@ -329,10 +480,121 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, + "x": 12, "y": 8 }, "hiddenSeries": false, + "id": 91, + "legend": { + "alignAsTable": true, + "avg": true, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": null, + "sortDesc": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "A", + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Queries Using Plan Cache OPS", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 0, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": null, + "description": "kv/tso request total by command type", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 15 + }, + "hiddenSeries": false, "id": 180, "legend": { "alignAsTable": true, @@ -381,15 +643,31 @@ "expr": "sum(rate(tidb_tikvclient_request_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", "hide": false, "interval": "", - "legendFormat": "total", + "legendFormat": "kv request total", "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "tso - cmd", + "refId": "C" + }, + { + "exemplar": true, + "expr": "sum(rate(pd_client_request_handle_requests_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "tso - request", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV Request OPS", + "title": "KV/TSO Request OPS", "tooltip": { "msResolution": false, "shared": true, @@ -435,7 +713,7 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": null, - "description": "TiDB plan cache hit total.", + "description": "TiDB current connection counts", "editable": true, "error": false, "fieldConfig": { @@ -449,10 +727,10 @@ "h": 7, "w": 12, "x": 12, - "y": 8 + "y": 15 }, "hiddenSeries": false, - "id": 91, + "id": 188, "legend": { "alignAsTable": true, "avg": true, @@ -463,8 +741,8 @@ "min": false, "rightSide": true, "show": true, - "sort": null, - "sortDesc": null, + "sort": "avg", + "sortDesc": true, "total": false, "values": true }, @@ -487,25 +765,42 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "interval": "", "intervalFactor": 2, - "legendFormat": "avg", + "legendFormat": "{{instance}}", "refId": "A", "step": 30 + }, + { + "exemplar": true, + "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "total", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m]))", + "hide": false, + "interval": "", + "legendFormat": "active connections", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Queries Using Plan Cache OPS", + "title": "Connection Count", "tooltip": { "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -560,7 +855,7 @@ "h": 7, "w": 12, "x": 0, - "y": 15 + "y": 22 }, "hiddenSeries": false, "id": 181, @@ -687,7 +982,7 @@ "h": 7, "w": 12, "x": 12, - "y": 15 + "y": 22 }, "hiddenSeries": false, "id": 182, @@ -810,7 +1105,7 @@ "h": 7, "w": 12, "x": 0, - "y": 22 + "y": 29 }, "hiddenSeries": false, "id": 80, @@ -824,6 +1119,8 @@ "min": false, "rightSide": true, "show": true, + "sort": "avg", + "sortDesc": true, "total": false, "values": true }, @@ -933,7 +1230,7 @@ "h": 7, "w": 12, "x": 12, - "y": 22 + "y": 29 }, "hiddenSeries": false, "id": 171, @@ -1070,7 +1367,7 @@ "h": 7, "w": 12, "x": 0, - "y": 29 + "y": 36 }, "hiddenSeries": false, "id": 156, @@ -1196,7 +1493,7 @@ "h": 7, "w": 12, "x": 12, - "y": 29 + "y": 36 }, "hiddenSeries": false, "id": 170, @@ -1309,7 +1606,7 @@ "h": 7, "w": 12, "x": 0, - "y": 36 + "y": 43 }, "hiddenSeries": false, "id": 169, @@ -1421,7 +1718,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of TiDB starts to wait for the TSO until received the TS result.", + "description": "The duration of a client starting to wait for the TS until received the TS result.", "editable": true, "error": false, "fieldConfig": { @@ -1435,7 +1732,7 @@ "h": 7, "w": 12, "x": 12, - "y": 36 + "y": 43 }, "hiddenSeries": false, "id": 77, @@ -1468,25 +1765,31 @@ "steppedLine": false, "targets": [ { + "exemplar": true, "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "hide": true, + "interval": "", "intervalFactor": 2, "legendFormat": "999", "refId": "A", "step": 10 }, { + "exemplar": true, "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", + "interval": "", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { + "exemplar": true, "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "hide": true, + "interval": "", "intervalFactor": 2, "legendFormat": "90", "refId": "C" @@ -1563,7 +1866,7 @@ "h": 11, "w": 12, "x": 0, - "y": 43 + "y": 50 }, "hiddenSeries": false, "id": 172, @@ -1673,7 +1976,7 @@ "h": 11, "w": 12, "x": 12, - "y": 43 + "y": 50 }, "hiddenSeries": false, "id": 173, @@ -1783,7 +2086,7 @@ "h": 7, "w": 12, "x": 0, - "y": 54 + "y": 61 }, "hiddenSeries": false, "id": 185, @@ -1901,7 +2204,7 @@ "h": 7, "w": 12, "x": 12, - "y": 54 + "y": 61 }, "hiddenSeries": false, "id": 183, @@ -2019,7 +2322,7 @@ "h": 8, "w": 12, "x": 0, - "y": 61 + "y": 68 }, "hiddenSeries": false, "id": 174, @@ -2137,7 +2440,7 @@ "h": 8, "w": 12, "x": 12, - "y": 61 + "y": 68 }, "hiddenSeries": false, "id": 177, @@ -2255,7 +2558,7 @@ "h": 8, "w": 12, "x": 0, - "y": 69 + "y": 76 }, "hiddenSeries": false, "id": 176, @@ -2373,7 +2676,7 @@ "h": 8, "w": 12, "x": 12, - "y": 69 + "y": 76 }, "hiddenSeries": false, "id": 186, diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 2c5ca4746b487..855fb1984fb5c 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -3180,14 +3180,14 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_packet_io_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_server_packet_io_bytes{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{type}}-rate", "refId": "A" }, { - "expr": "sum(tidb_server_packet_io_bytes_sum{tidb_cluster=\"$tidb_cluster\"}) by (type)", + "expr": "sum(tidb_server_packet_io_bytes{tidb_cluster=\"$tidb_cluster\"}) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{type}}-total", @@ -5475,6 +5475,99 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 53 + }, + "hiddenSeries": false, + "id": 229, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": false, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": {}, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(irate(tidb_tikvclient_prewrite_assertion_count[30s])) by (type)", + "interval": "", + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Assertion", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:369", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:370", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -9178,6 +9271,14 @@ "legendFormat": "{{type}}", "refId": "A", "step": 10 + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_ddl_batch_add_idx_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "add index worker", + "refId": "B", + "step": 10 } ], "thresholds": [], @@ -12744,7 +12845,7 @@ "alignLevel": null } }, - { + { "aliasColors": {}, "bars": false, "dashLength": 10, diff --git a/metrics/grafana/tidb_runtime.json b/metrics/grafana/tidb_runtime.json index 63136f16d0b02..6cfd02e3636f1 100644 --- a/metrics/grafana/tidb_runtime.json +++ b/metrics/grafana/tidb_runtime.json @@ -982,136 +982,190 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "Number of requests in each batch", + "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 36 + "y": 29 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, + "hiddenSeries": false, "id": 23, "legend": { - "show": false + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_batch_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", - "format": "heatmap", + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le))", + "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{le}}", + "legendFormat": "99", "refId": "A", "step": 40 } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, "title": "Requests Batch Size", "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "showHistogram": false + "values": [] }, - "type": "heatmap", - "xAxis": { + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", "show": true }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", + { + "format": "short", + "label": null, "logBase": 1, "max": null, "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "description": "Number of requests in queue", + "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 36 + "y": 29 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, + "hiddenSeries": false, "id": 24, "legend": { - "show": false + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "reverseYBuckets": false, + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_batch_pending_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_pending_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le))", "format": "heatmap", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{le}}", + "legendFormat": "99", "refId": "A", "step": 40 } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, "title": "Pending Requests", "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "showHistogram": false + "values": [] }, - "type": "heatmap", - "xAxis": { + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", "show": true }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", + { + "format": "short", + "label": null, "logBase": 1, "max": null, "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, diff --git a/metrics/grafana/tidb_summary.json b/metrics/grafana/tidb_summary.json index 0d12628605f2e..b04e9e505fa90 100644 --- a/metrics/grafana/tidb_summary.json +++ b/metrics/grafana/tidb_summary.json @@ -1,3114 +1,1799 @@ { - "__inputs": [ - { - "name": "DS_TEST-CLUSTER", - "label": "test-cluster", - "description": "", - "type": "datasource", - "pluginId": "prometheus", - "pluginName": "Prometheus" - } - ], - "__requires": [ - { - "type": "grafana", - "id": "grafana", - "name": "Grafana", - "version": "6.1.6" - }, - { - "type": "panel", - "id": "graph", - "name": "Graph", - "version": "" - }, - { - "type": "datasource", - "id": "prometheus", - "name": "Prometheus", - "version": "1.0.0" - } - ], - "annotations": { - "list": [ + "__inputs": [ { - "builtIn": 1, - "datasource": "${DS_TEST-CLUSTER}", - "enable": true, - "hide": true, - "iconColor": "rgba(0, 211, 255, 1)", - "name": "Annotations & Alerts", - "type": "dashboard" + "description": "", + "label": "test-cluster", + "name": "DS_TEST-CLUSTER", + "pluginId": "prometheus", + "pluginName": "Prometheus", + "type": "datasource" } - ] - }, - "editable": true, - "gnetId": null, - "graphTooltip": 1, - "id": null, - "links": [], - "panels": [ - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 0 - }, - "id": 140, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "cacheTimeout": null, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB uptime since the last restart.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 1 - }, - "id": 184, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pluginVersion": "6.1.6", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "total", - "fill": 0, - "lines": false - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"})", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Uptime", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "dtdurations", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB current connection counts.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 1 - }, - "id": 8, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "total", - "fill": 0, - "lines": false - } - ], - "spaceLength": 10, - "stack": true, - "steppedLine": false, - "targets": [ - { - "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - }, - { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "total", - "refId": "B", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Connection Count", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB CPU usage calculated with process CPU running seconds.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 8 - }, - "id": 168, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "total", - "fill": 0, - "lines": false - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "rate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[1m])", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CPU Usage", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 1, - "format": "percentunit", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB process rss memory usage.\nTiDB heap memory size in use.", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 8 - }, - "id": 3, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "process-{{instance}}", - "refId": "A" - }, - { - "expr": "go_memstats_heap_sys_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "HeapSys-{{instance}}", - "refId": "B" - }, - { - "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "HeapInuse-{{instance}}", - "refId": "C" - }, - { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "HeapAlloc-{{instance}}", - "refId": "D" - }, - { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "HeapIdle-{{instance}}", - "refId": "E" - }, - { - "expr": "go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", - "hide": true, - "interval": "", - "legendFormat": "HeapReleased-{{instance}}", - "refId": "F" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Memory Usage", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": "", - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": "", - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Server", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 1 - }, - "id": 138, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB query durations by histogram buckets with different percents.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 2 - }, - "id": 80, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": false, - "hideZero": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95", - "refId": "C" - }, - { - "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s])) / sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s]))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "refId": "D" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Duration", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "TiDB failed query statistics by query type.", - "fill": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 2 - }, - "id": 137, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": 250, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": " {{type}}-{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Failed Query OPS", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "MySQL command processing numbers per second. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 9 - }, - "id": 42, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": 250, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "maxPerRow": 1, - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "query {{result}}", - "refId": "A", - "step": 60 - }, - { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 2, - "legendFormat": "yesterday", - "refId": "B", - "step": 90 - }, - { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m]))", - "format": "time_series", - "hide": true, - "instant": false, - "intervalFactor": 2, - "legendFormat": "ideal CPS", - "refId": "C", - "step": 60 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Command Per Second", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB query total statistics including both successful and failed ones.", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 9 - }, - "id": 2, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": 250, - "sort": "max", - "sortDesc": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "maxPerRow": 1, - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "total", - "lines": false - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}} ", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CPS By Instance", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "TiDB statement statistics.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, + ], + "__requires": [ ], + "annotations": { + "list": [ ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 1, + "hideControls": false, + "id": null, + "links": [ ], + "panels": [ + { + "collapse": true, + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, "x": 0, - "y": 16 - }, - "id": 21, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 + "y": 0 + }, + "id": 2, + "panels": [ + { + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB uptime since the last restart.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 3, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Uptime", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "total", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "QPS", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB current connection counts.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 4, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + }, + { + "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "total", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Connection Count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "MySQL command statistics by command type. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", - "fill": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 16 - }, - "id": 189, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": 250, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": " {{type}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CPS by CMD", - "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB CPU usage calculated with process CPU running seconds.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 5, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "rate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[1m])", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "CPU Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB process rss memory usage.TiDB heap memory size in use.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 6, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "process-{{instance}}", + "refId": "A" + }, + { + "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "HeapInuse-{{instance}}", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Memory Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Query Summary", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 + ], + "repeat": null, + "repeatIteration": null, + "repeatRowId": null, + "showTitle": true, + "title": "Server", + "titleSize": "h6", + "type": "row" }, - "id": 142, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "The time cost of parsing SQL to AST", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, + { + "collapse": true, + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, "x": 0, - "y": 3 - }, - "id": 156, - "interval": "", - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "99", - "refId": "A", - "step": 30 + "y": 0 + }, + "id": 7, + "panels": [ + { + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB query durations by histogram buckets with different percents.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 8, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95", + "refId": "B" + }, + { + "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s])) / sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "C" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Parse Duration", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": null, - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB failed query statistics by query type.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 9, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-{{instance}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Failed Query OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "The time cost of building the query plan", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 3 - }, - "id": 154, - "interval": "", - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "99", - "refId": "A", - "step": 30 - }, - { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Compile Duration", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": null, - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "The time cost of executing the SQL which does not include the time to get the results of the query .", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 10 - }, - "id": 169, - "interval": "", - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "instant": false, - "intervalFactor": 2, - "legendFormat": "99", - "refId": "A", - "step": 30 + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "MySQL command processing numbers per second. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 10, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "query {{result}}", + "refId": "A" + }, + { + "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "yesterday", + "refId": "B" + }, + { + "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "ideal CPS", + "refId": "C" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Command Per Second", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "95", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Execution Duration", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": null, - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB query total statistics including both successful and failed ones.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 11, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "CPS By Instance", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "TiDB plan cache hit total.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 10 - }, - "id": 91, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": null, - "sortDesc": null, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 30 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Queries Using Plan Cache OPS", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB statement statistics.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 12, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + }, + { + "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "total", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "QPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "MySQL command statistics by command type. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 13, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "CPS by CMD", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Query Detail", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 + ], + "repeat": null, + "repeatIteration": null, + "repeatRowId": null, + "showTitle": true, + "title": "Query Summary", + "titleSize": "h6", + "type": "row" }, - "id": 141, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB transaction processing counts by type.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, + { + "collapse": true, + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, "x": 0, - "y": 4 - }, - "id": 69, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}-{{txn_mode}}", - "refId": "A", - "step": 10 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "TPS", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "y": 0 + }, + "id": 14, + "panels": [ + { + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time cost of parsing SQL to AST", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 15, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Parse Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "Bucketed histogram of transaction execution durations, including retry.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 4 - }, - "id": 72, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99-{{txn_mode}}", - "refId": "A" + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time cost of building the query plan", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 16, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Compile Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "95-{{txn_mode}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "80-{{txn_mode}}", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Duration", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The max TiDB statements numbers within one transaction.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 11 - }, - "id": 74, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(1, sum(rate(tidb_session_transaction_statement_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "max", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Max Transaction Statement Num", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The max TiDB transaction retry count.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 11 - }, - "id": 67, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(1.0, sum(rate(tidb_session_retry_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "max", - "refId": "A", - "step": 10 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Max Transaction Retry Num", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time cost of executing the SQL which does not include the time to get the results of the query.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 17, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95", + "refId": "B" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Execution Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB plan cache hit total.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 18, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Queries Using Plan Cache OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Transaction", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 4 + ], + "repeat": null, + "repeatIteration": null, + "repeatRowId": null, + "showTitle": true, + "title": "Query Detail", + "titleSize": "h6", + "type": "row" }, - "id": 145, - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "KV request durations by store (TiKV). It contains requests that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 5 - }, - "id": 48, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": "max", - "sortDesc": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le, store))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "store-{{store}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV Request Duration 99 by store", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "KV request durations by request type. It contains requests that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 5 - }, - "id": 30, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": "max", - "sortDesc": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le,type))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV Request Duration 99 by type", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "KV request count by request type. It contains requests that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 5 - }, - "id": 172, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_tikvclient_request_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV Request OPS", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB total kv transaction counts. It contains transactions that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 12 - }, - "id": 4, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "KV Transaction OPS", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The max writes bytes of the transaction. It contains transactions that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 12 - }, - "id": 34, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": "avg", - "sortDesc": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Max Transaction Write Size Bytes", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 2, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The max writes kv count of the transaction. It contains transactions that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 12 - }, - "id": 33, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sort": "avg", - "sortDesc": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_kv_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "B", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Max Transaction Write KV Num", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 2, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The max writes regions of the transaction. It contains transactions that are executed automatically by the internal background.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 0, - "y": 19 - }, - "id": 44, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_regions_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A", - "step": 40 - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Max Transaction Regions Num", - "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of TiDB starts to wait for the TSO until received the TS result.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 19 - }, - "id": 77, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "999", - "refId": "A", - "step": 10 - }, - { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "90", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "PD TSO Wait Duration", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of a client starting to wait for the TS until received the TS result.", - "editable": true, - "error": false, - "fill": 1, - "grid": {}, - "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 19 - }, - "id": 78, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "999", - "refId": "A", - "step": 10 - }, - { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "90", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "PD TSO RPC Duration", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "cumulative" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB auto-ID requests per second including single table/global auto-ID processing and single table auto-ID rebase processing.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, + { + "collapse": true, + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, "x": 0, - "y": 26 - }, - "id": 50, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(rate(tidb_autoid_operation_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "AutoID QPS", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": 0, - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true + "y": 0 + }, + "id": 19, + "panels": [ + { + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB transaction processing counts by type.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 20, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}-{{txn_mode}}", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "TPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB auto-ID requests durations.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 26 - }, - "id": 51, - "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "histogram_quantile(0.99, sum(rate(tidb_autoid_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "99", - "refId": "B" + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bucketed histogram of transaction execution durations, including retry.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 21, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "99-{{txn_mode}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "95-{{txn_mode}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "80-{{txn_mode}}", + "refId": "C" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Transaction Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "expr": "sum(rate(tidb_autoid_operation_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_autoid_operation_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "avg", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "AutoID Duration", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "decimals": null, - "format": "s", - "label": "", - "logBase": 1, - "max": null, - "min": "0", - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The max TiDB statements numbers within one transaction.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 0, + "y": 0 + }, + "id": 22, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1, sum(rate(tidb_session_transaction_statement_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "max", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Max Transaction Statement Num", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] }, { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true + "aliasColors": { }, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The max TiDB transaction retry count.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 6, + "w": 12, + "x": 12, + "y": 0 + }, + "id": 23, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [ ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "seriesOverrides": [ ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(1.0, sum(rate(tidb_session_retry_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "max", + "refId": "A" + } + ], + "thresholds": [ ], + "timeFrom": null, + "timeShift": null, + "title": "Max Transaction Retry Num", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [ ] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ] } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "Write Slow", - "type": "row" - } - ], - "refresh": "30s", - "schemaVersion": 18, - "style": "dark", - "tags": [], - "templating": { - "list": [ - { - "allValue": null, - "current": { - }, - "datasource": "${DS_TEST-CLUSTER}", - "hide": 2, - "includeAll": false, - "label": "tidb_cluster", - "multi": false, - "name": "tidb_cluster", - "options": [ - - ], - "query": "label_values(pd_cluster_status, tidb_cluster)", - "refresh": 2, - "regex": "", - "sort": 1, - "tagValuesQuery": "", - "tags": [ - - ], - "tagsQuery": "", - "type": "query", - "useTags": false + ], + "repeat": null, + "repeatIteration": null, + "repeatRowId": null, + "showTitle": true, + "title": "Transaction", + "titleSize": "h6", + "type": "row" } - ] - }, - "time": { - "from": "now-1h", - "to": "now" - }, - "timepicker": { - "refresh_intervals": [ - "5s", - "10s", - "30s", - "1m", - "5m", - "15m", - "30m", - "1h", - "2h", - "1d" - ], - "time_options": [ - "5m", - "15m", - "1h", - "6h", - "12h", - "24h", - "2d", - "7d", - "30d" - ] - }, - "timezone": "browser", - "title": "Test-Cluster-TiDB-Summary", - "uid": "000000012", - "version": 1 -} \ No newline at end of file + ], + "refresh": "30s", + "rows": [ ], + "schemaVersion": 14, + "style": "dark", + "tags": [ ], + "templating": { + "list": [ + { + "allValue": null, + "current": { }, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "tidb_cluster", + "multi": false, + "name": "tidb_cluster", + "options": [ ], + "query": "label_values(pd_cluster_status, tidb_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [ ], + "tagsQuery": "", + "type": "query", + "useTags": false + } + ] + }, + "time": { + "from": "now-1h", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "browser", + "title": "Test-Cluster-TiDB-Summary", + "version": 0 +} diff --git a/metrics/grafana/tidb_summary.jsonnet b/metrics/grafana/tidb_summary.jsonnet new file mode 100644 index 0000000000000..c090afede4111 --- /dev/null +++ b/metrics/grafana/tidb_summary.jsonnet @@ -0,0 +1,442 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +local grafana = import 'grafonnet/grafana.libsonnet'; +local dashboard = grafana.dashboard; +local row = grafana.row; +local graphPanel = grafana.graphPanel; +local prometheus = grafana.prometheus; +local template = grafana.template; + +local myNameFlag = 'DS_TEST-CLUSTER'; +local myDS = '${' + myNameFlag + '}'; + +// A new dashboard +local newDash = dashboard.new( + title='Test-Cluster-TiDB-Summary', + editable=true, + graphTooltip='shared_crosshair', + refresh='30s', + time_from='now-1h', +) +.addInput( + name=myNameFlag, + label='test-cluster', + type='datasource', + pluginId='prometheus', + pluginName='Prometheus', +) +.addTemplate( + // Default template for tidb-cloud + template.new( + allValues=null, + current=null, + datasource=myDS, + hide='all', + includeAll=false, + label='tidb_cluster', + multi=false, + name='tidb_cluster', + query='label_values(pd_cluster_status, tidb_cluster)', + refresh='time', + regex='', + sort=1, + tagValuesQuery='', + ) +); + +// Server row and its panels +local serverRow = row.new(collapse=true, title='Server'); +local uptimeP = graphPanel.new( + title='Uptime', + datasource=myDS, + legend_rightSide=true, + format='s', + description='TiDB uptime since the last restart.', +) +.addTarget( + prometheus.target( + 'time() - process_start_time_seconds{tidb_cluster="$tidb_cluster", job="tidb"}', + legendFormat='{{instance}}', + ) +); + +local connectionP = graphPanel.new( + title='Connection Count', + datasource=myDS, + legend_rightSide=true, + description='TiDB current connection counts.', + format='short', + stack=true, +) +.addTarget( + prometheus.target( + 'tidb_server_connections{tidb_cluster="$tidb_cluster"}', + legendFormat='{{instance}}', + ) +) +.addTarget( + prometheus.target( + 'sum(tidb_server_connections{tidb_cluster="$tidb_cluster"})', + legendFormat='total', + ) +); + +local cpuP = graphPanel.new( + title='CPU Usage', + datasource=myDS, + legend_rightSide=true, + description='TiDB CPU usage calculated with process CPU running seconds.', + format='percentunit', +) +.addTarget( + prometheus.target( + 'rate(process_cpu_seconds_total{tidb_cluster="$tidb_cluster", job="tidb"}[1m])', + legendFormat='{{instance}}', + ) +); + +local memP = graphPanel.new( + title='Memory Usage', + datasource=myDS, + legend_rightSide=true, + description='TiDB process rss memory usage.TiDB heap memory size in use.', + format='bytes', +) +.addTarget( + prometheus.target( + 'process_resident_memory_bytes{tidb_cluster="$tidb_cluster", job="tidb"}', + legendFormat='process-{{instance}}', + ) +) +.addTarget( + prometheus.target( + 'go_memstats_heap_inuse_bytes{tidb_cluster="$tidb_cluster", job="tidb"}', + legendFormat='HeapInuse-{{instance}}', + ) +); + +// Query Summary +local queryRow = row.new(collapse=true, title='Query Summary'); +local durationP = graphPanel.new( + title='Duration', + datasource=myDS, + legend_rightSide=true, + description='TiDB query durations by histogram buckets with different percents.', + format='s', +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type!="internal"}[1m])) by (le))', + legendFormat='99', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type!="internal"}[1m])) by (le))', + legendFormat='95', + ) +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster="$tidb_cluster", sql_type!="internal"}[30s])) / sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster="$tidb_cluster", sql_type!="internal"}[30s]))', + legendFormat='avg', + ) +); + +local failedP = graphPanel.new( + title='Failed Query OPS', + datasource=myDS, + legend_rightSide=true, + description='TiDB failed query statistics by query type.', + format='short', +) +.addTarget( + prometheus.target( + 'sum(increase(tidb_server_execute_error_total{tidb_cluster="$tidb_cluster"}[1m])) by (type, instance)', + legendFormat='{{type}}-{{instance}}', + ) +); + +local cpsP = graphPanel.new( + title='Command Per Second', + datasource=myDS, + legend_rightSide=true, + description='MySQL command processing numbers per second. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html', + format='short', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_query_total{tidb_cluster="$tidb_cluster"}[1m])) by (result)', + legendFormat='query {{result}}', + ) +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_query_total{tidb_cluster="$tidb_cluster", result="OK"}[1m] offset 1d))', + legendFormat='yesterday', + hide=true, + ) +) +.addTarget( + prometheus.target( + 'sum(tidb_server_connections{tidb_cluster="$tidb_cluster"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster="$tidb_cluster"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster="$tidb_cluster"}[1m]))', + legendFormat='ideal CPS', + hide=true, + ) +); + +local cpsByInstP = graphPanel.new( + title='CPS By Instance', + datasource=myDS, + legend_rightSide=true, + description='TiDB query total statistics including both successful and failed ones.', + format='short', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_query_total{tidb_cluster="$tidb_cluster"}[1m])) by (instance)', + legendFormat='{{instance}}', + ) +); + +local qpsP = graphPanel.new( + title='QPS', + datasource=myDS, + legend_rightSide=true, + description='TiDB statement statistics.', + format='short', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_executor_statement_total{tidb_cluster="$tidb_cluster"}[1m])) by (type)', + legendFormat='{{type}}', + ) +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_executor_statement_total{tidb_cluster="$tidb_cluster"}[1m]))', + legendFormat='total', + ) +); + +local cpsByCMDP = graphPanel.new( + title='CPS by CMD', + datasource=myDS, + legend_rightSide=true, + description='MySQL command statistics by command type. See https://dev.mysql.com/doc/internals/en/text-protocol.html and https://dev.mysql.com/doc/internals/en/prepared-statements.html', + format='short', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_query_total{tidb_cluster="$tidb_cluster"}[1m])) by (type)', + legendFormat='{{type}}', + ) +); + +// Query Detail row and its panels +local queryDetailRow = row.new(collapse=true, title='Query Detail'); +local parseP = graphPanel.new( + title='Parse Duration', + datasource=myDS, + legend_rightSide=true, + format='s', + description='The time cost of parsing SQL to AST', +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.99, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='99', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='95', + ) +); + +local compileP = graphPanel.new( + title='Compile Duration', + datasource=myDS, + legend_rightSide=true, + description='The time cost of building the query plan', + format='s', +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.99, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='99', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='95', + ) +); + +local exeP = graphPanel.new( + title='Execution Duration', + datasource=myDS, + legend_rightSide=true, + description='The time cost of executing the SQL which does not include the time to get the results of the query.', + format='s', +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.99, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='99', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le))', + legendFormat='95', + ) +); + +local planCacheP = graphPanel.new( + title='Queries Using Plan Cache OPS', + datasource=myDS, + legend_rightSide=true, + description='TiDB plan cache hit total.', + format='short', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_server_plan_cache_total{tidb_cluster="$tidb_cluster"}[1m])) by (type)', + legendFormat='{{type}}', + ) +); + +// Transaction row and its panels +local txnRow = row.new(collapse=true, title='Transaction'); +local tpsP = graphPanel.new( + title='TPS', + datasource=myDS, + legend_rightSide=true, + format='short', + description='TiDB transaction processing counts by type.', +) +.addTarget( + prometheus.target( + 'sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster="$tidb_cluster"}[1m])) by (type, txn_mode)', + legendFormat='{{type}}-{{txn_mode}}', + ) +); + +local txnDurationP = graphPanel.new( + title='Transaction Duration', + datasource=myDS, + legend_rightSide=true, + description='Bucketed histogram of transaction execution durations, including retry.', + format='s', +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le, txn_mode))', + legendFormat='99-{{txn_mode}}', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le, txn_mode))', + legendFormat='95-{{txn_mode}}', + ) +) +.addTarget( + prometheus.target( + 'histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster="$tidb_cluster", sql_type="general"}[1m])) by (le, txn_mode))', + legendFormat='80-{{txn_mode}}', + ) +); + +local maxTxnStmtP = graphPanel.new( + title='Max Transaction Statement Num', + datasource=myDS, + legend_rightSide=true, + description='The max TiDB statements numbers within one transaction.', + format='short', +) +.addTarget( + prometheus.target( + 'histogram_quantile(1, sum(rate(tidb_session_transaction_statement_num_bucket{tidb_cluster="$tidb_cluster"}[30s])) by (le))', + legendFormat='max', + ) +); + +local maxTxnRetryP = graphPanel.new( + title='Max Transaction Retry Num', + datasource=myDS, + legend_rightSide=true, + description='The max TiDB transaction retry count.', + format='short', +) +.addTarget( + prometheus.target( + 'histogram_quantile(1.0, sum(rate(tidb_session_retry_num_bucket{tidb_cluster="$tidb_cluster"}[30s])) by (le))', + legendFormat='max', + ) +); + +// Merge together. +local panelW = 12; +local panelH = 6; +local rowW = 24; +local rowH = 1; + +local rowPos = {x:0, y:0, w:rowW, h:rowH}; +local leftPanelPos = {x:0, y:0, w:panelW, h:panelH}; +local rightPanelPos = {x:panelW, y:0, w:panelW, h:panelH}; + +newDash +.addPanel( + serverRow + .addPanel(uptimeP, gridPos=leftPanelPos) + .addPanel(connectionP, gridPos=rightPanelPos) + .addPanel(cpuP, gridPos=leftPanelPos) + .addPanel(memP, gridPos=rightPanelPos) + , + gridPos=rowPos +) +.addPanel( + queryRow + .addPanel(durationP, gridPos=leftPanelPos) + .addPanel(failedP, gridPos=rightPanelPos) + .addPanel(cpsP, gridPos=leftPanelPos) + .addPanel(cpsByInstP, gridPos=rightPanelPos) + .addPanel(qpsP, gridPos=leftPanelPos) + .addPanel(cpsByCMDP, gridPos=rightPanelPos) + , + gridPos=rowPos +) +.addPanel( + queryDetailRow + .addPanel(parseP, gridPos=leftPanelPos) + .addPanel(compileP, gridPos=rightPanelPos) + .addPanel(exeP, gridPos=leftPanelPos) + .addPanel(planCacheP, gridPos=rightPanelPos) + , + gridPos=rowPos +) +.addPanel( + txnRow + .addPanel(tpsP, gridPos=leftPanelPos) + .addPanel(txnDurationP, gridPos=rightPanelPos) + .addPanel(maxTxnStmtP, gridPos=leftPanelPos) + .addPanel(maxTxnRetryP, gridPos=rightPanelPos) + , + gridPos=rowPos +) diff --git a/metrics/metrics.go b/metrics/metrics.go index b24eb7b152750..9ee4656d57cd1 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -112,7 +112,7 @@ func RegisterMetrics() { prometheus.MustRegister(PanicCounter) prometheus.MustRegister(PlanCacheCounter) prometheus.MustRegister(PseudoEstimation) - prometheus.MustRegister(PacketIOHistogram) + prometheus.MustRegister(PacketIOCounter) prometheus.MustRegister(QueryDurationHistogram) prometheus.MustRegister(QueryTotalCounter) prometheus.MustRegister(SchemaLeaseErrorCounter) @@ -162,6 +162,7 @@ func RegisterMetrics() { prometheus.MustRegister(TopSQLReportDataHistogram) prometheus.MustRegister(PDApiExecutionHistogram) prometheus.MustRegister(CPUProfileCounter) + prometheus.MustRegister(ReadFromTableCacheCounter) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/metrics/server.go b/metrics/server.go index 440a833e6f03e..e373144e97297 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -27,13 +27,12 @@ var ( // Metrics var ( - PacketIOHistogram = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ + PacketIOCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ Namespace: "tidb", Subsystem: "server", Name: "packet_io_bytes", - Help: "Bucketed histogram of packet IO bytes.", - Buckets: prometheus.ExponentialBuckets(4, 4, 21), // 4Bytes ~ 4TB + Help: "Counters of packet IO bytes.", }, []string{LblType}) QueryDurationHistogram = prometheus.NewHistogramVec( @@ -129,6 +128,15 @@ var ( Help: "Counter of query using plan cache.", }, []string{LblType}) + ReadFromTableCacheCounter = prometheus.NewCounter( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "read_from_tablecache_total", + Help: "Counter of query read from table cache.", + }, + ) + HandShakeErrorCounter = prometheus.NewCounter( prometheus.CounterOpts{ Namespace: "tidb", diff --git a/metrics/telemetry.go b/metrics/telemetry.go index 7db666d8dc737..e98f5a9f5d6d0 100644 --- a/metrics/telemetry.go +++ b/metrics/telemetry.go @@ -52,11 +52,11 @@ type CTEUsageCounter struct { // Sub returns the difference of two counters. func (c CTEUsageCounter) Sub(rhs CTEUsageCounter) CTEUsageCounter { - new := CTEUsageCounter{} - new.NonRecursiveCTEUsed = c.NonRecursiveCTEUsed - rhs.NonRecursiveCTEUsed - new.RecursiveUsed = c.RecursiveUsed - rhs.RecursiveUsed - new.NonCTEUsed = c.NonCTEUsed - rhs.NonCTEUsed - return new + return CTEUsageCounter{ + NonRecursiveCTEUsed: c.NonRecursiveCTEUsed - rhs.NonRecursiveCTEUsed, + RecursiveUsed: c.RecursiveUsed - rhs.RecursiveUsed, + NonCTEUsed: c.NonCTEUsed - rhs.NonCTEUsed, + } } // GetCTECounter gets the TxnCommitCounter. diff --git a/owner/fail_test.go b/owner/fail_test.go index 600a9131bca34..4f93fe278c002 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "google.golang.org/grpc" ) diff --git a/owner/main_test.go b/owner/main_test.go index ec1eeb75cf4a5..ba902fed7556b 100644 --- a/owner/main_test.go +++ b/owner/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), } goleak.VerifyTestMain(m, opts...) } diff --git a/owner/manager.go b/owner/manager.go index 93a0b760c7b81..3bed53a1907b5 100644 --- a/owner/manager.go +++ b/owner/manager.go @@ -31,10 +31,10 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" - "go.etcd.io/etcd/mvcc/mvccpb" + "go.etcd.io/etcd/api/v3/mvccpb" + "go.etcd.io/etcd/api/v3/v3rpc/rpctypes" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" "go.uber.org/zap" "google.golang.org/grpc" ) diff --git a/owner/manager_test.go b/owner/manager_test.go index af458965a7da3..f2c1f5da0584c 100644 --- a/owner/manager_test.go +++ b/owner/manager_test.go @@ -29,9 +29,9 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/logutil" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/integration" + clientv3 "go.etcd.io/etcd/client/v3" + "go.etcd.io/etcd/client/v3/concurrency" + "go.etcd.io/etcd/tests/v3/integration" goctx "golang.org/x/net/context" ) @@ -41,6 +41,7 @@ func TestSingle(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) store, err := mockstore.NewMockStore() require.NoError(t, err) @@ -100,6 +101,7 @@ func TestCluster(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) originalTTL := owner.ManagerSessionTTL owner.ManagerSessionTTL = 3 diff --git a/parser/README.md b/parser/README.md index c5bc9b1ce603f..509a7a8a465b8 100644 --- a/parser/README.md +++ b/parser/README.md @@ -7,13 +7,13 @@ The goal of this project is to build a Golang parser that is fully compatible with MySQL syntax, easy to extend, and high performance. Currently, features supported by parser are as follows: -- Highly compatible with MySQL: it supports almost all features of MySQL. For the complete details, see [parser.y](https://github.com/pingcap/parser/blob/master/parser.y) and [hintparser.y](https://github.com/pingcap/parser/blob/master/hintparser.y). +- Highly compatible with MySQL: it supports almost all features of MySQL. For the complete details, see [parser.y](https://github.com/pingcap/tidb/blob/master/parser/parser.y) and [hintparser.y](https://github.com/pingcap/tidb/blob/master/parser/hintparser.y). - Extensible: adding a new syntax requires only a few lines of Yacc and Golang code changes. As an example, see [PR-680](https://github.com/pingcap/parser/pull/680/files). - Good performance: the parser is generated by goyacc in a bottom-up approach. It is efficient to build an AST tree with a state machine. ## How to use it -Please read the [quickstart](https://github.com/pingcap/parser/blob/master/docs/quickstart.md). +Please read the [quickstart](https://github.com/pingcap/tidb/blob/master/parser/docs/quickstart.md). ## Future @@ -24,7 +24,7 @@ Please read the [quickstart](https://github.com/pingcap/parser/blob/master/docs/ ## Getting Help -- [GitHub Issue](https://github.com/pingcap/parser/issues) +- [GitHub Issue](https://github.com/pingcap/tidb/issues) - [Stack Overflow](https://stackoverflow.com/questions/tagged/tidb) - [User Group (Chinese)](https://asktug.com/) @@ -50,8 +50,6 @@ found you are one of the users but not listed here: Contributions are welcomed and greatly appreciated. See [Contribution Guide](https://github.com/pingcap/community/blob/master/contributors/README.md) for details on submitting patches and the contribution workflow. -Here is how to [update parser for TiDB](https://github.com/pingcap/parser/blob/master/docs/update-parser-for-tidb.md). - ## Acknowledgments Thanks [cznic](https://github.com/cznic) for providing some great open-source tools. diff --git a/parser/ast/ddl.go b/parser/ast/ddl.go index 63e27cb8ebb9a..6ab941275c14b 100644 --- a/parser/ast/ddl.go +++ b/parser/ast/ddl.go @@ -73,14 +73,16 @@ const ( DatabaseOptionCharset DatabaseOptionCollate DatabaseOptionEncryption + DatabaseSetTiFlashReplica DatabaseOptionPlacementPolicy = DatabaseOptionType(PlacementOptionPolicy) ) // DatabaseOption represents database option. type DatabaseOption struct { - Tp DatabaseOptionType - Value string - UintValue uint64 + Tp DatabaseOptionType + Value string + UintValue uint64 + TiFlashReplica *TiFlashReplicaSpec } // Restore implements Node interface. @@ -105,6 +107,19 @@ func (n *DatabaseOption) Restore(ctx *format.RestoreCtx) error { StrValue: n.Value, } return placementOpt.Restore(ctx) + case DatabaseSetTiFlashReplica: + ctx.WriteKeyWord("SET TIFLASH REPLICA ") + ctx.WritePlainf("%d", n.TiFlashReplica.Count) + if len(n.TiFlashReplica.Labels) == 0 { + break + } + ctx.WriteKeyWord(" LOCATION LABELS ") + for i, v := range n.TiFlashReplica.Labels { + if i > 0 { + ctx.WritePlain(", ") + } + ctx.WriteString(v) + } default: return errors.Errorf("invalid DatabaseOptionType: %d", n.Tp) } diff --git a/parser/ast/functions.go b/parser/ast/functions.go index 7775f3dbc2029..9779f7ba446b6 100644 --- a/parser/ast/functions.go +++ b/parser/ast/functions.go @@ -289,6 +289,7 @@ const ( UUIDToBin = "uuid_to_bin" BinToUUID = "bin_to_uuid" VitessHash = "vitess_hash" + TiDBShard = "tidb_shard" // get_lock() and release_lock() is parsed but do nothing. // It is used for preventing error in Ruby's activerecord migrations. GetLock = "get_lock" diff --git a/parser/docs/quickstart.md b/parser/docs/quickstart.md index f116b240b43b5..a3832f50efdaa 100644 --- a/parser/docs/quickstart.md +++ b/parser/docs/quickstart.md @@ -17,18 +17,20 @@ go mod init colx && touch main.go ## Import Dependencies -First of all, you need to use `go get` to fetch the dependencies through git hash. The git hashes are available in [release page](https://github.com/pingcap/parser/releases). Take `v4.0.2` as an example: +First, you need to use `go get` to fetch the dependencies through git hash. The git hashes are available in [release page](https://github.com/pingcap/tidb/releases). Take `v5.3.0` as an example: ```bash -go get -v github.com/pingcap/parser@3a18f1e +go get -v github.com/pingcap/tidb/parser@4a1b2e9 ``` > **NOTE** +> +> The parser was merged into TiDB repo since v5.3.0. So you can only choose version v5.3.0 or higher in this TiDB repo. > -> You may want to use advanced API on expressions (a kind of AST node), such as numbers, string literals, booleans, nulls, etc. It is strongly recommended to use the `types` package in TiDB repo with the following command: +> You may want to use advanced API on expressions (a kind of AST node), such as numbers, string literals, booleans, nulls, etc. It is strongly recommended using the `types` package in TiDB repo with the following command: > > ```bash -> go get -v github.com/pingcap/tidb/types/parser_driver@328b6d0 +> go get -v github.com/pingcap/tidb/types/parser_driver@4a1b2e9 > ``` > and import it in your golang source code: > ```go @@ -48,17 +50,18 @@ Now, open `main.go` with your favorite editor, and start coding! ## Parse SQL text To convert a SQL text to an AST tree, you need to: -1. Use the [`parser.New()`](https://pkg.go.dev/github.com/pingcap/parser?tab=doc#New) function to instantiate a parser, and -2. Invoke the method [`Parse(sql, charset, collation)`](https://pkg.go.dev/github.com/pingcap/parser?tab=doc#Parser.Parse) on the parser. +1. Use the [`parser.New()`](https://pkg.go.dev/github.com/pingcap/tidb/parser?tab=doc#New) function to instantiate a parser, and +2. Invoke the method [`Parse(sql, charset, collation)`](https://pkg.go.dev/github.com/pingcap/tidb/parser?tab=doc#Parser.Parse) on the parser. ```go package main import ( "fmt" - "github.com/pingcap/parser" - "github.com/pingcap/parser/ast" - _ "github.com/pingcap/parser/test_driver" + + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + _ "github.com/pingcap/tidb/parser/test_driver" ) func parse(sql string) (*ast.StmtNode, error) { @@ -80,6 +83,7 @@ func main() { } fmt.Printf("%v\n", *astNode) } + ``` Test the parser by running the following command: @@ -99,19 +103,19 @@ If the parser runs properly, you should get a result like this: > Here are a few things you might want to know: > - To use a parser, a `parser_driver` is required. It decides how to parse the basic data types in SQL. > -> You can use [`github.com/pingcap/parser/test_driver`](https://pkg.go.dev/github.com/pingcap/parser/test_driver) as the `parser_driver` for test. Again, if you need advanced features, please use the `parser_driver` in TiDB (run `go get -v github.com/pingcap/tidb/types/parser_driver@328b6d0` and import it). +> You can use [`github.com/pingcap/tidb/parser/test_driver`](https://pkg.go.dev/github.com/pingcap/tidb/parser/test_driver) as the `parser_driver` for test. Again, if you need advanced features, please use the `parser_driver` in TiDB (run `go get -v github.com/pingcap/tidb/types/parser_driver@4a1b2e9` and import it). > - The instantiated parser object is not goroutine safe. It is better to keep it in a single goroutine. > - The instantiated parser object is not lightweight. It is better to reuse it if possible. -> - The 2nd and 3rd arguments of [`parser.Parse()`](https://pkg.go.dev/github.com/pingcap/parser?tab=doc#Parser.Parse) are charset and collation respectively. If you pass an empty string into it, a default value is chosen. +> - The 2nd and 3rd arguments of [`parser.Parse()`](https://pkg.go.dev/github.com/pingcap/tidb/parser?tab=doc#Parser.Parse) are charset and collation respectively. If you pass an empty string into it, a default value is chosen. ## Traverse AST Nodes Now you get the AST tree root of a SQL statement. It is time to extract the column names by traverse. -Parser implements the interface [`ast.Node`](https://pkg.go.dev/github.com/pingcap/parser/ast?tab=doc#Node) for each kind of AST node, such as SelectStmt, TableName, ColumnName. [`ast.Node`](https://pkg.go.dev/github.com/pingcap/parser/ast?tab=doc#Node) provides a method `Accept(v Visitor) (node Node, ok bool)` to allow any struct that has implemented [`ast.Visitor`](https://pkg.go.dev/github.com/pingcap/parser/ast?tab=doc#Visitor) to traverse itself. +Parser implements the interface [`ast.Node`](https://pkg.go.dev/github.com/pingcap/tidb/parser/ast?tab=doc#Node) for each kind of AST node, such as SelectStmt, TableName, ColumnName. [`ast.Node`](https://pkg.go.dev/github.com/pingcap/tidb/parser/ast?tab=doc#Node) provides a method `Accept(v Visitor) (node Node, ok bool)` to allow any struct that has implemented [`ast.Visitor`](https://pkg.go.dev/github.com/pingcap/tidb/parser/ast?tab=doc#Visitor) to traverse itself. -[`ast.Visitor`](https://pkg.go.dev/github.com/pingcap/parser/ast?tab=doc#Visitor) is defined as follows: +[`ast.Visitor`](https://pkg.go.dev/github.com/pingcap/tidb/parser/ast?tab=doc#Visitor) is defined as follows: ```go type Visitor interface { Enter(n Node) (node Node, skipChildren bool) diff --git a/parser/docs/update-parser-for-tidb.md b/parser/docs/update-parser-for-tidb.md deleted file mode 100644 index c1b4d80088387..0000000000000 --- a/parser/docs/update-parser-for-tidb.md +++ /dev/null @@ -1,42 +0,0 @@ -# How to update parser for TiDB - -Assuming that you want to file a PR (pull request) to TiDB, and your PR includes a change in the parser, follow these steps to update the parser in TiDB. - -## Step 1: Make changes in your parser repository - -Fork this repository to your own account and commit the changes to your repository. - -> **Note:** -> -> - Don't forget to run `make test` before you commit! -> - Make sure `parser.go` is updated. - -Suppose the forked repository is `https://github.com/your-repo/parser`. - -## Step 2: Make your parser changes take effect in TiDB and run CI - -1. In your TiDB repository, execute the `replace` instruction to make your parser changes take effect: - - ``` - GO111MODULE=on go mod edit -replace github.com/pingcap/parser=github.com/your-repo/parser@your-branch - ``` - -2. `make dev` to run CI in TiDB. - -3. File a PR to TiDB. - -## Step 3: Merge the PR about the parser to this repository - -File a PR to this repository. **Link the related PR in TiDB in your PR description or comment.** - -This PR will be reviewed, and if everything goes well, it will be merged. - -## Step 4: Update TiDB to use the latest parser - -In your TiDB pull request, modify the `go.mod` file manually or use this command: - -``` -GO111MODULE=on go get -u github.com/pingcap/parser@master -``` - -Make sure the `replace` instruction is changed back to the `require` instruction and the version is the latest. diff --git a/parser/misc.go b/parser/misc.go index 730ffa9312900..97085e40eccc6 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -797,9 +797,10 @@ var tokenMap = map[string]int{ "WAIT": wait, } -// See https://dev.mysql.com/doc/refman/5.7/en/function-resolution.html for details +// See https://dev.mysql.com/doc/refman/5.7/en/function-resolution.html for details. +// ADDDATE, SESSION_USER, SUBDATE, and SYSTEM_USER are exceptions because they are actually recognized as +// identifiers even in `create table adddate (a int)`. var btFuncTokenMap = map[string]int{ - "ADDDATE": builtinAddDate, "BIT_AND": builtinBitAnd, "BIT_OR": builtinBitOr, "BIT_XOR": builtinBitXor, @@ -818,17 +819,14 @@ var btFuncTokenMap = map[string]int{ "MIN": builtinMin, "NOW": builtinNow, "POSITION": builtinPosition, - "SESSION_USER": builtinUser, "STD": builtinStddevPop, "STDDEV": builtinStddevPop, "STDDEV_POP": builtinStddevPop, "STDDEV_SAMP": builtinStddevSamp, - "SUBDATE": builtinSubDate, "SUBSTR": builtinSubstring, "SUBSTRING": builtinSubstring, "SUM": builtinSum, "SYSDATE": builtinSysDate, - "SYSTEM_USER": builtinUser, "TRANSLATE": builtinTranslate, "TRIM": builtinTrim, "VARIANCE": builtinVarPop, diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 12433473a6712..f939edddb5d0a 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -222,13 +222,28 @@ type DDLReorgMeta struct { SQLMode mysql.SQLMode `json:"sql_mode"` Warnings map[errors.ErrorID]*terror.Error `json:"warnings"` WarningsCount map[errors.ErrorID]int64 `json:"warnings_count"` - Location *TimeZone `json:"time_zone"` + Location *TimeZoneLocation `json:"location"` } -// TimeZone represents a single time zone. -type TimeZone struct { - Name string - Offset int // seconds east of UTC +// TimeZoneLocation represents a single time zone. +type TimeZoneLocation struct { + Name string `json:"name"` + Offset int `json:"offset"` // seconds east of UTC + location *time.Location +} + +func (tz *TimeZoneLocation) GetLocation() (*time.Location, error) { + if tz.location != nil { + return tz.location, nil + } + + var err error + if tz.Offset == 0 { + tz.location, err = time.LoadLocation(tz.Name) + } else { + tz.location = time.FixedZone(tz.Name, tz.Offset) + } + return tz.location, err } // NewDDLReorgMeta new a DDLReorgMeta. @@ -290,6 +305,9 @@ type Job struct { // Priority is only used to set the operation priority of adding indices. Priority int `json:"priority"` + + // SeqNum is the total order in all DDLs, it's used to identify the order of DDL. + SeqNum uint64 `json:"seq_num"` } // FinishTableJob is called when a job is finished. diff --git a/parser/model/model_test.go b/parser/model/model_test.go index f3e9ec2a400d7..041f825a21a1d 100644 --- a/parser/model/model_test.go +++ b/parser/model/model_test.go @@ -158,7 +158,7 @@ func TestJobCodec(t *testing.T) { BinlogInfo: &HistoryInfo{}, Args: []interface{}{NewCIStr("a"), A{Name: "abc"}}, ReorgMeta: &DDLReorgMeta{ - Location: &TimeZone{Name: tzName, Offset: tzOffset}, + Location: &TimeZoneLocation{Name: tzName, Offset: tzOffset}, }, } job.BinlogInfo.AddDBInfo(123, &DBInfo{ID: 1, Name: NewCIStr("test_history_db")}) @@ -439,3 +439,30 @@ func TestPlacementSettingsString(t *testing.T) { } require.Equal(t, "CONSTRAINTS=\"{+us-east-1:1,+us-east-2:1}\" VOTERS=3 FOLLOWERS=2 LEARNERS=1", settings.String()) } + +func TestLocation(t *testing.T) { + // test offset = 0 + loc := &TimeZoneLocation{} + nLoc, err := loc.GetLocation() + require.NoError(t, err) + require.Equal(t, nLoc.String(), "UTC") + // test loc.location != nil + loc.Name = "Asia/Shanghai" + nLoc, err = loc.GetLocation() + require.NoError(t, err) + require.Equal(t, nLoc.String(), "UTC") + // timezone +05:00 + loc1 := &TimeZoneLocation{Name: "UTC", Offset: 18000} + loc1Byte, err := json.Marshal(loc1) + require.NoError(t, err) + loc2 := &TimeZoneLocation{} + err = json.Unmarshal(loc1Byte, loc2) + require.NoError(t, err) + require.Equal(t, loc2.Offset, loc1.Offset) + require.Equal(t, loc2.Name, loc1.Name) + nLoc, err = loc2.GetLocation() + require.NoError(t, err) + require.Equal(t, nLoc.String(), "UTC") + location := time.FixedZone("UTC", loc1.Offset) + require.Equal(t, nLoc, location) +} diff --git a/parser/parser.go b/parser/parser.go index f414e11c4227c..e765333b8a9c1 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -54,7 +54,7 @@ type yyXError struct { } const ( - yyDefault = 58103 + yyDefault = 58101 yyEOFCode = 57344 account = 57573 action = 57574 @@ -72,7 +72,7 @@ const ( analyze = 57362 and = 57363 andand = 57354 - andnot = 58064 + andnot = 58062 any = 57581 approxCountDistinct = 57909 approxPercentile = 57910 @@ -80,7 +80,7 @@ const ( asc = 57365 ascii = 57582 asof = 57347 - assignmentEq = 58065 + assignmentEq = 58063 attributes = 57583 autoIdCache = 57588 autoIncrement = 57589 @@ -100,7 +100,7 @@ const ( bindings = 57600 binlog = 57601 bitAnd = 57911 - bitLit = 58063 + bitLit = 58061 bitOr = 57912 bitType = 57602 bitXor = 57913 @@ -113,35 +113,33 @@ const ( briefType = 57915 btree = 57606 buckets = 57992 - builtinAddDate = 58030 - builtinApproxCountDistinct = 58036 - builtinApproxPercentile = 58037 - builtinBitAnd = 58031 - builtinBitOr = 58032 - builtinBitXor = 58033 - builtinCast = 58034 - builtinCount = 58035 - builtinCurDate = 58038 - builtinCurTime = 58039 - builtinDateAdd = 58040 - builtinDateSub = 58041 - builtinExtract = 58042 - builtinGroupConcat = 58043 - builtinMax = 58044 - builtinMin = 58045 - builtinNow = 58046 - builtinPosition = 58047 - builtinStddevPop = 58052 - builtinStddevSamp = 58053 - builtinSubDate = 58048 - builtinSubstring = 58049 - builtinSum = 58050 - builtinSysDate = 58051 - builtinTranslate = 58054 - builtinTrim = 58055 - builtinUser = 58056 - builtinVarPop = 58057 - builtinVarSamp = 58058 + builtinApproxCountDistinct = 58035 + builtinApproxPercentile = 58036 + builtinBitAnd = 58030 + builtinBitOr = 58031 + builtinBitXor = 58032 + builtinCast = 58033 + builtinCount = 58034 + builtinCurDate = 58037 + builtinCurTime = 58038 + builtinDateAdd = 58039 + builtinDateSub = 58040 + builtinExtract = 58041 + builtinGroupConcat = 58042 + builtinMax = 58043 + builtinMin = 58044 + builtinNow = 58045 + builtinPosition = 58046 + builtinStddevPop = 58050 + builtinStddevSamp = 58051 + builtinSubstring = 58047 + builtinSum = 58048 + builtinSysDate = 58049 + builtinTranslate = 58052 + builtinTrim = 58053 + builtinUser = 58054 + builtinVarPop = 58055 + builtinVarSamp = 58056 builtins = 57993 by = 57371 byteType = 57607 @@ -195,7 +193,7 @@ const ( correlation = 57998 cpu = 57636 create = 57383 - createTableSelect = 58087 + createTableSelect = 58085 cross = 57384 csvBackslashEscape = 57637 csvDelimiter = 57638 @@ -227,7 +225,7 @@ const ( daySecond = 57396 ddl = 57999 deallocate = 57651 - decLit = 58060 + decLit = 58058 decimalType = 57397 defaultKwd = 57398 definer = 57652 @@ -257,7 +255,7 @@ const ( duplicate = 57659 dynamic = 57660 elseKwd = 57410 - empty = 58078 + empty = 58076 enable = 57661 enclosed = 57411 encryption = 57662 @@ -266,7 +264,7 @@ const ( engine = 57665 engines = 57666 enum = 57667 - eq = 58066 + eq = 58064 yyErrCode = 57345 errorKwd = 57668 escape = 57669 @@ -295,7 +293,7 @@ const ( firstValue = 57418 fixed = 57683 flashback = 57927 - floatLit = 58059 + floatLit = 58057 floatType = 57419 flush = 57684 follower = 57928 @@ -310,7 +308,7 @@ const ( full = 57687 fulltext = 57424 function = 57688 - ge = 58067 + ge = 58065 general = 57689 generated = 57425 getFormat = 57931 @@ -323,10 +321,10 @@ const ( hash = 57692 having = 57429 help = 57693 - hexLit = 58062 + hexLit = 58060 highPriority = 57430 - higherThanComma = 58102 - higherThanParenthese = 58096 + higherThanComma = 58100 + higherThanParenthese = 58094 hintComment = 57353 histogram = 57694 histogramsInFlight = 58019 @@ -353,7 +351,7 @@ const ( inplace = 57934 insert = 57446 insertMethod = 57705 - insertValues = 58085 + insertValues = 58083 instance = 57706 instant = 57935 int1Type = 57448 @@ -361,7 +359,7 @@ const ( int3Type = 57450 int4Type = 57451 int8Type = 57452 - intLit = 58061 + intLit = 58059 intType = 57447 integerType = 57440 internal = 57936 @@ -382,8 +380,8 @@ const ( jsonArrayagg = 57937 jsonObjectAgg = 57938 jsonType = 57713 - jss = 58069 - juss = 58070 + jss = 58067 + juss = 58068 key = 57454 keyBlockSize = 57714 keys = 57455 @@ -395,7 +393,7 @@ const ( lastBackup = 57718 lastValue = 57458 lastval = 57719 - le = 58068 + le = 58066 lead = 57459 leader = 57939 leaderConstraints = 57940 @@ -423,25 +421,25 @@ const ( longblobType = 57470 longtextType = 57471 lowPriority = 57472 - lowerThanCharsetKwd = 58088 - lowerThanComma = 58101 - lowerThanCreateTableSelect = 58086 - lowerThanEq = 58098 - lowerThanFunction = 58093 - lowerThanInsertValues = 58084 - lowerThanKey = 58089 - lowerThanLocal = 58090 - lowerThanNot = 58100 - lowerThanOn = 58097 - lowerThanParenthese = 58095 - lowerThanRemove = 58091 - lowerThanSelectOpt = 58079 - lowerThanSelectStmt = 58083 - lowerThanSetKeyword = 58082 - lowerThanStringLitToken = 58081 - lowerThanValueKeyword = 58080 - lowerThenOrder = 58092 - lsh = 58071 + lowerThanCharsetKwd = 58086 + lowerThanComma = 58099 + lowerThanCreateTableSelect = 58084 + lowerThanEq = 58096 + lowerThanFunction = 58091 + lowerThanInsertValues = 58082 + lowerThanKey = 58087 + lowerThanLocal = 58088 + lowerThanNot = 58098 + lowerThanOn = 58095 + lowerThanParenthese = 58093 + lowerThanRemove = 58089 + lowerThanSelectOpt = 58077 + lowerThanSelectStmt = 58081 + lowerThanSetKeyword = 58080 + lowerThanStringLitToken = 58079 + lowerThanValueKeyword = 58078 + lowerThenOrder = 58090 + lsh = 58069 master = 57727 match = 57473 max = 57945 @@ -474,9 +472,9 @@ const ( national = 57746 natural = 57572 ncharType = 57747 - neg = 58099 - neq = 58072 - neqSynonym = 58073 + neg = 58097 + neq = 58070 + neqSynonym = 58071 never = 57748 next = 57749 next_row_id = 57933 @@ -493,13 +491,13 @@ const ( nonclustered = 57757 none = 57758 not = 57481 - not2 = 58077 + not2 = 58075 now = 57946 nowait = 57759 nthValue = 57483 ntile = 57484 null = 57485 - nulleq = 58074 + nulleq = 58072 nulls = 57761 numericType = 57486 nvarcharType = 57760 @@ -527,7 +525,7 @@ const ( over = 57495 packKeys = 57769 pageSym = 57770 - paramMarker = 58075 + paramMarker = 58073 parser = 57771 partial = 57772 partition = 57496 @@ -616,7 +614,7 @@ const ( rowFormat = 57820 rowNumber = 57519 rows = 57518 - rsh = 58076 + rsh = 58074 rtree = 57821 running = 57956 s3 = 57957 @@ -717,7 +715,7 @@ const ( systemTime = 57873 tableChecksum = 57874 tableKwd = 57534 - tableRefPriority = 58094 + tableRefPriority = 58092 tableSample = 57535 tables = 57875 tablespace = 57876 @@ -816,566 +814,566 @@ const ( zerofill = 57571 yyMaxDepth = 200 - yyTabOfs = -2458 + yyTabOfs = -2459 ) var ( yyXLAT = map[int]int{ - 57344: 0, // $end (2166x) - 59: 1, // ';' (2165x) - 57802: 2, // remove (1826x) - 57803: 3, // reorganize (1826x) - 57625: 4, // comment (1762x) - 57864: 5, // storage (1738x) - 57589: 6, // autoIncrement (1727x) - 44: 7, // ',' (1646x) - 57682: 8, // first (1626x) - 57576: 9, // after (1624x) - 57831: 10, // serial (1620x) - 57590: 11, // autoRandom (1619x) - 57622: 12, // columnFormat (1619x) - 57775: 13, // password (1594x) - 57613: 14, // charsetKwd (1587x) - 57615: 15, // checksum (1580x) - 57948: 16, // placement (1573x) - 57714: 17, // keyBlockSize (1562x) - 57876: 18, // tablespace (1559x) - 57665: 19, // engine (1554x) - 57647: 20, // data (1552x) - 57662: 21, // encryption (1552x) - 57705: 22, // insertMethod (1550x) - 57732: 23, // maxRows (1550x) - 57739: 24, // minRows (1550x) - 57754: 25, // nodegroup (1550x) - 57632: 26, // connection (1542x) - 57591: 27, // autoRandomBase (1539x) - 58016: 28, // statsBuckets (1537x) - 58018: 29, // statsTopN (1537x) - 57588: 30, // autoIdCache (1536x) - 57593: 31, // avgRowLength (1536x) - 57630: 32, // compression (1536x) - 57653: 33, // delayKeyWrite (1536x) - 57769: 34, // packKeys (1536x) - 57782: 35, // preSplitRegions (1536x) - 57820: 36, // rowFormat (1536x) - 57824: 37, // secondaryEngine (1536x) - 57835: 38, // shardRowIDBits (1536x) - 57860: 39, // statsAutoRecalc (1536x) - 57586: 40, // statsColChoice (1536x) - 57587: 41, // statsColList (1536x) - 57861: 42, // statsPersistent (1536x) - 57862: 43, // statsSamplePages (1536x) - 57585: 44, // statsSampleRate (1536x) - 57874: 45, // tableChecksum (1536x) - 57573: 46, // account (1483x) - 57814: 47, // resume (1473x) - 57839: 48, // signed (1473x) - 57845: 49, // snapshot (1472x) - 41: 50, // ')' (1471x) - 57594: 51, // backend (1471x) - 57614: 52, // checkpoint (1471x) - 57631: 53, // concurrency (1471x) - 57637: 54, // csvBackslashEscape (1471x) - 57638: 55, // csvDelimiter (1471x) - 57639: 56, // csvHeader (1471x) - 57640: 57, // csvNotNull (1471x) - 57641: 58, // csvNull (1471x) - 57642: 59, // csvSeparator (1471x) - 57643: 60, // csvTrimLastSeparators (1471x) - 57718: 61, // lastBackup (1471x) - 57764: 62, // onDuplicate (1471x) - 57765: 63, // online (1471x) - 57797: 64, // rateLimit (1471x) - 57828: 65, // sendCredentialsToTiKV (1471x) - 57842: 66, // skipSchemaFiles (1471x) - 57865: 67, // strictFormat (1471x) - 57881: 68, // tikvImporter (1471x) - 57889: 69, // truncate (1468x) - 57751: 70, // no (1467x) - 57859: 71, // start (1465x) - 57608: 72, // cache (1462x) - 57752: 73, // nocache (1461x) - 57646: 74, // cycle (1460x) - 57741: 75, // minValue (1460x) - 57702: 76, // increment (1459x) - 57753: 77, // nocycle (1459x) - 57755: 78, // nomaxvalue (1459x) - 57756: 79, // nominvalue (1459x) - 57811: 80, // restart (1457x) - 57579: 81, // algorithm (1456x) - 57884: 82, // tp (1456x) - 57645: 83, // clustered (1455x) - 57707: 84, // invisible (1455x) - 57757: 85, // nonclustered (1455x) - 58028: 86, // regions (1455x) - 57900: 87, // visible (1455x) - 57918: 88, // constraints (1448x) - 57929: 89, // followerConstraints (1448x) - 57930: 90, // followers (1448x) - 57940: 91, // leaderConstraints (1448x) - 57942: 92, // learnerConstraints (1448x) - 57943: 93, // learners (1448x) - 57953: 94, // primaryRegion (1448x) - 57958: 95, // schedule (1448x) - 57989: 96, // voterConstraints (1448x) - 57990: 97, // voters (1448x) - 57623: 98, // columns (1447x) - 57899: 99, // view (1447x) - 57867: 100, // subpartition (1443x) - 57582: 101, // ascii (1442x) - 57607: 102, // byteType (1442x) - 57774: 103, // partitions (1442x) - 57893: 104, // unicodeSym (1442x) - 57906: 105, // yearType (1442x) - 57650: 106, // day (1441x) - 57680: 107, // fields (1441x) - 57823: 108, // second (1440x) - 57858: 109, // sqlTsiYear (1440x) - 57875: 110, // tables (1440x) - 57697: 111, // hour (1439x) - 57738: 112, // microsecond (1439x) - 57740: 113, // minute (1439x) - 57744: 114, // month (1439x) - 57793: 115, // quarter (1439x) - 57851: 116, // sqlTsiDay (1439x) - 57852: 117, // sqlTsiHour (1439x) - 57853: 118, // sqlTsiMinute (1439x) - 57854: 119, // sqlTsiMonth (1439x) - 57855: 120, // sqlTsiQuarter (1439x) - 57856: 121, // sqlTsiSecond (1439x) - 57857: 122, // sqlTsiWeek (1439x) - 57902: 123, // week (1439x) - 57829: 124, // separator (1438x) - 57863: 125, // status (1438x) - 57730: 126, // maxConnectionsPerHour (1437x) - 57731: 127, // maxQueriesPerHour (1437x) - 57733: 128, // maxUpdatesPerHour (1437x) - 57734: 129, // maxUserConnections (1437x) - 57783: 130, // preceding (1437x) - 57616: 131, // cipher (1436x) - 57700: 132, // importKwd (1436x) - 57712: 133, // issuer (1436x) - 57822: 134, // san (1436x) - 57866: 135, // subject (1436x) - 57723: 136, // local (1435x) - 57841: 137, // skip (1435x) - 57600: 138, // bindings (1434x) - 57652: 139, // definer (1434x) - 57692: 140, // hash (1434x) - 57698: 141, // identified (1434x) - 57726: 142, // logs (1434x) - 57795: 143, // query (1434x) - 57810: 144, // respect (1434x) - 57626: 145, // commit (1433x) - 57644: 146, // current (1433x) - 57664: 147, // enforced (1433x) - 57685: 148, // following (1433x) - 57759: 149, // nowait (1433x) - 57766: 150, // only (1433x) - 57817: 151, // rollback (1433x) - 57897: 152, // value (1433x) - 57597: 153, // begin (1432x) - 57599: 154, // binding (1432x) - 57663: 155, // end (1432x) - 57690: 156, // global (1432x) - 57933: 157, // next_row_id (1432x) - 57781: 158, // policy (1432x) - 57952: 159, // predicate (1432x) - 57877: 160, // temporary (1432x) - 57890: 161, // unbounded (1432x) - 57895: 162, // user (1432x) - 57346: 163, // identifier (1431x) - 57763: 164, // offset (1431x) - 57950: 165, // planCache (1431x) - 57784: 166, // prepare (1431x) - 57816: 167, // role (1431x) - 57894: 168, // unknown (1431x) - 57907: 169, // wait (1431x) - 57606: 170, // btree (1430x) - 57648: 171, // datetimeType (1430x) - 57649: 172, // dateType (1430x) - 57683: 173, // fixed (1430x) - 57711: 174, // isolation (1430x) - 57713: 175, // jsonType (1430x) - 57728: 176, // max_idxnum (1430x) - 57736: 177, // memory (1430x) - 57762: 178, // off (1430x) - 57768: 179, // optional (1430x) - 57777: 180, // per_db (1430x) - 57786: 181, // privileges (1430x) - 57809: 182, // required (1430x) - 57821: 183, // rtree (1430x) - 57956: 184, // running (1430x) - 58011: 185, // sampleRate (1430x) - 57830: 186, // sequence (1430x) - 57833: 187, // session (1430x) - 57844: 188, // slow (1430x) - 57883: 189, // timeType (1430x) - 57896: 190, // validation (1430x) - 57898: 191, // variables (1430x) - 57583: 192, // attributes (1429x) - 57655: 193, // disable (1429x) - 57659: 194, // duplicate (1429x) - 57660: 195, // dynamic (1429x) - 57661: 196, // enable (1429x) - 57668: 197, // errorKwd (1429x) - 57684: 198, // flush (1429x) - 57687: 199, // full (1429x) - 57699: 200, // identSQLErrors (1429x) - 57725: 201, // location (1429x) - 57735: 202, // mb (1429x) - 57742: 203, // mode (1429x) - 57748: 204, // never (1429x) - 57949: 205, // plan (1429x) - 57780: 206, // plugins (1429x) - 57788: 207, // processlist (1429x) - 57799: 208, // recover (1429x) - 57804: 209, // repair (1429x) - 57805: 210, // repeatable (1429x) - 58012: 211, // statistics (1429x) - 57868: 212, // subpartitions (1429x) - 58022: 213, // tidb (1429x) - 57882: 214, // timestampType (1429x) - 57904: 215, // without (1429x) - 57991: 216, // admin (1428x) - 57595: 217, // backup (1428x) - 57601: 218, // binlog (1428x) - 57603: 219, // block (1428x) - 57604: 220, // booleanType (1428x) - 57992: 221, // buckets (1428x) - 57995: 222, // cardinality (1428x) - 57612: 223, // chain (1428x) - 57619: 224, // clientErrorsSummary (1428x) - 57996: 225, // cmSketch (1428x) - 57620: 226, // coalesce (1428x) - 57628: 227, // compact (1428x) - 57629: 228, // compressed (1428x) - 57635: 229, // context (1428x) - 57917: 230, // copyKwd (1428x) - 57998: 231, // correlation (1428x) - 57636: 232, // cpu (1428x) - 57651: 233, // deallocate (1428x) - 58000: 234, // dependency (1428x) - 57654: 235, // directory (1428x) - 57656: 236, // discard (1428x) - 57657: 237, // disk (1428x) - 57658: 238, // do (1428x) - 58002: 239, // drainer (1428x) - 57673: 240, // exchange (1428x) - 57675: 241, // execute (1428x) - 57676: 242, // expansion (1428x) - 57927: 243, // flashback (1428x) - 57689: 244, // general (1428x) - 57693: 245, // help (1428x) - 57694: 246, // histogram (1428x) - 57696: 247, // hosts (1428x) - 57934: 248, // inplace (1428x) - 57706: 249, // instance (1428x) - 57935: 250, // instant (1428x) - 57710: 251, // ipc (1428x) - 58004: 252, // job (1428x) - 58003: 253, // jobs (1428x) - 57715: 254, // labels (1428x) - 57724: 255, // locked (1428x) - 57743: 256, // modify (1428x) - 57749: 257, // next (1428x) - 58005: 258, // nodeID (1428x) - 58006: 259, // nodeState (1428x) - 57761: 260, // nulls (1428x) - 57770: 261, // pageSym (1428x) - 58009: 262, // pump (1428x) - 57792: 263, // purge (1428x) - 57798: 264, // rebuild (1428x) - 57800: 265, // redundant (1428x) - 57801: 266, // reload (1428x) - 57812: 267, // restore (1428x) - 57818: 268, // routine (1428x) - 57957: 269, // s3 (1428x) - 58010: 270, // samples (1428x) - 57825: 271, // secondaryLoad (1428x) - 57826: 272, // secondaryUnload (1428x) - 57836: 273, // share (1428x) - 57838: 274, // shutdown (1428x) - 57847: 275, // source (1428x) - 58025: 276, // split (1428x) - 58013: 277, // stats (1428x) - 57584: 278, // statsOptions (1428x) - 57964: 279, // stop (1428x) - 57870: 280, // swaps (1428x) - 57974: 281, // tokudbDefault (1428x) - 57975: 282, // tokudbFast (1428x) - 57976: 283, // tokudbLzma (1428x) - 57977: 284, // tokudbQuickLZ (1428x) - 57979: 285, // tokudbSmall (1428x) - 57978: 286, // tokudbSnappy (1428x) - 57980: 287, // tokudbUncompressed (1428x) - 57981: 288, // tokudbZlib (1428x) - 58024: 289, // topn (1428x) - 57885: 290, // trace (1428x) - 57574: 291, // action (1427x) - 57575: 292, // advise (1427x) - 57577: 293, // against (1427x) - 57578: 294, // ago (1427x) - 57580: 295, // always (1427x) - 57596: 296, // backups (1427x) - 57598: 297, // bernoulli (1427x) - 57602: 298, // bitType (1427x) - 57605: 299, // boolType (1427x) - 57915: 300, // briefType (1427x) - 57993: 301, // builtins (1427x) - 57994: 302, // cancel (1427x) - 57609: 303, // capture (1427x) - 57610: 304, // cascaded (1427x) - 57611: 305, // causal (1427x) - 57617: 306, // cleanup (1427x) - 57618: 307, // client (1427x) - 57621: 308, // collation (1427x) - 57997: 309, // columnStatsUsage (1427x) - 57627: 310, // committed (1427x) - 57624: 311, // config (1427x) - 57633: 312, // consistency (1427x) - 57634: 313, // consistent (1427x) - 57999: 314, // ddl (1427x) - 58001: 315, // depth (1427x) - 57922: 316, // dotType (1427x) - 57923: 317, // dump (1427x) - 57666: 318, // engines (1427x) - 57667: 319, // enum (1427x) - 57671: 320, // events (1427x) - 57672: 321, // evolve (1427x) - 57677: 322, // expire (1427x) - 57925: 323, // exprPushdownBlacklist (1427x) - 57678: 324, // extended (1427x) - 57679: 325, // faultsSym (1427x) - 57686: 326, // format (1427x) - 57688: 327, // function (1427x) - 57691: 328, // grants (1427x) - 58019: 329, // histogramsInFlight (1427x) - 57695: 330, // history (1427x) - 57701: 331, // imports (1427x) - 57703: 332, // incremental (1427x) - 57704: 333, // indexes (1427x) - 57936: 334, // internal (1427x) - 57708: 335, // invoker (1427x) - 57709: 336, // io (1427x) - 57716: 337, // language (1427x) - 57717: 338, // last (1427x) - 57720: 339, // less (1427x) - 57721: 340, // level (1427x) - 57722: 341, // list (1427x) - 57727: 342, // master (1427x) - 57729: 343, // max_minutes (1427x) - 57737: 344, // merge (1427x) - 57746: 345, // national (1427x) - 57747: 346, // ncharType (1427x) - 57750: 347, // nextval (1427x) - 57758: 348, // none (1427x) - 57760: 349, // nvarcharType (1427x) - 57767: 350, // open (1427x) - 58007: 351, // optimistic (1427x) - 57947: 352, // optRuleBlacklist (1427x) - 57771: 353, // parser (1427x) - 57772: 354, // partial (1427x) - 57773: 355, // partitioning (1427x) - 57778: 356, // per_table (1427x) - 57776: 357, // percent (1427x) - 58008: 358, // pessimistic (1427x) - 57785: 359, // preserve (1427x) - 57789: 360, // profile (1427x) - 57790: 361, // profiles (1427x) - 57794: 362, // queries (1427x) - 57954: 363, // recent (1427x) - 58029: 364, // region (1427x) - 57955: 365, // replayer (1427x) - 57806: 366, // replica (1427x) - 58027: 367, // reset (1427x) - 57813: 368, // restores (1427x) - 57827: 369, // security (1427x) - 57832: 370, // serializable (1427x) - 57840: 371, // simple (1427x) - 57843: 372, // slave (1427x) - 58017: 373, // statsHealthy (1427x) - 58015: 374, // statsHistograms (1427x) - 58014: 375, // statsMeta (1427x) - 57965: 376, // strict (1427x) - 57871: 377, // switchesSym (1427x) - 57872: 378, // system (1427x) - 57873: 379, // systemTime (1427x) - 57970: 380, // target (1427x) - 58021: 381, // telemetryID (1427x) - 57878: 382, // temptable (1427x) - 57879: 383, // textType (1427x) - 57880: 384, // than (1427x) - 58023: 385, // tiFlash (1427x) - 57973: 386, // tls (1427x) - 57982: 387, // top (1427x) - 57886: 388, // traditional (1427x) - 57887: 389, // transaction (1427x) - 57888: 390, // triggers (1427x) - 57891: 391, // uncommitted (1427x) - 57892: 392, // undefined (1427x) - 57987: 393, // verboseType (1427x) - 57901: 394, // warnings (1427x) - 58026: 395, // width (1427x) - 57905: 396, // x509 (1427x) - 57908: 397, // addDate (1426x) - 57581: 398, // any (1426x) - 57909: 399, // approxCountDistinct (1426x) - 57910: 400, // approxPercentile (1426x) - 57592: 401, // avg (1426x) - 57911: 402, // bitAnd (1426x) - 57912: 403, // bitOr (1426x) - 57913: 404, // bitXor (1426x) - 57914: 405, // bound (1426x) - 57916: 406, // cast (1426x) - 57919: 407, // curTime (1426x) - 57920: 408, // dateAdd (1426x) - 57921: 409, // dateSub (1426x) - 57669: 410, // escape (1426x) - 57670: 411, // event (1426x) - 57924: 412, // exact (1426x) - 57674: 413, // exclusive (1426x) - 57926: 414, // extract (1426x) - 57681: 415, // file (1426x) - 57928: 416, // follower (1426x) - 57931: 417, // getFormat (1426x) - 57932: 418, // groupConcat (1426x) - 57937: 419, // jsonArrayagg (1426x) - 57938: 420, // jsonObjectAgg (1426x) - 57719: 421, // lastval (1426x) - 57939: 422, // leader (1426x) - 57941: 423, // learner (1426x) - 57945: 424, // max (1426x) - 57944: 425, // min (1426x) - 57745: 426, // names (1426x) - 57946: 427, // now (1426x) - 57951: 428, // position (1426x) - 57787: 429, // process (1426x) - 57791: 430, // proxy (1426x) - 57796: 431, // quick (1426x) - 57807: 432, // replicas (1426x) - 57808: 433, // replication (1426x) - 57815: 434, // reverse (1426x) - 57819: 435, // rowCount (1426x) - 57834: 436, // setval (1426x) - 57837: 437, // shared (1426x) - 57846: 438, // some (1426x) - 57848: 439, // sqlBufferResult (1426x) - 57849: 440, // sqlCache (1426x) - 57850: 441, // sqlNoCache (1426x) - 57959: 442, // staleness (1426x) - 57960: 443, // std (1426x) - 57961: 444, // stddev (1426x) - 57962: 445, // stddevPop (1426x) - 57963: 446, // stddevSamp (1426x) - 57966: 447, // strong (1426x) - 57967: 448, // subDate (1426x) - 57969: 449, // substring (1426x) - 57968: 450, // sum (1426x) - 57869: 451, // super (1426x) - 58020: 452, // telemetry (1426x) - 57971: 453, // timestampAdd (1426x) - 57972: 454, // timestampDiff (1426x) - 57983: 455, // trim (1426x) - 57984: 456, // variance (1426x) - 57985: 457, // varPop (1426x) - 57986: 458, // varSamp (1426x) - 57988: 459, // voter (1426x) - 57903: 460, // weightString (1426x) - 57488: 461, // on (1360x) + 57344: 0, // $end (2170x) + 59: 1, // ';' (2169x) + 57802: 2, // remove (1828x) + 57803: 3, // reorganize (1828x) + 57625: 4, // comment (1764x) + 57864: 5, // storage (1740x) + 57589: 6, // autoIncrement (1729x) + 44: 7, // ',' (1648x) + 57682: 8, // first (1628x) + 57576: 9, // after (1626x) + 57831: 10, // serial (1622x) + 57590: 11, // autoRandom (1621x) + 57622: 12, // columnFormat (1621x) + 57775: 13, // password (1596x) + 57613: 14, // charsetKwd (1594x) + 57615: 15, // checksum (1582x) + 57948: 16, // placement (1580x) + 57714: 17, // keyBlockSize (1564x) + 57876: 18, // tablespace (1561x) + 57662: 19, // encryption (1559x) + 57665: 20, // engine (1556x) + 57647: 21, // data (1554x) + 57705: 22, // insertMethod (1552x) + 57732: 23, // maxRows (1552x) + 57739: 24, // minRows (1552x) + 57754: 25, // nodegroup (1552x) + 57632: 26, // connection (1544x) + 57591: 27, // autoRandomBase (1541x) + 58016: 28, // statsBuckets (1539x) + 58018: 29, // statsTopN (1539x) + 57588: 30, // autoIdCache (1538x) + 57593: 31, // avgRowLength (1538x) + 57630: 32, // compression (1538x) + 57653: 33, // delayKeyWrite (1538x) + 57769: 34, // packKeys (1538x) + 57782: 35, // preSplitRegions (1538x) + 57820: 36, // rowFormat (1538x) + 57824: 37, // secondaryEngine (1538x) + 57835: 38, // shardRowIDBits (1538x) + 57860: 39, // statsAutoRecalc (1538x) + 57586: 40, // statsColChoice (1538x) + 57587: 41, // statsColList (1538x) + 57861: 42, // statsPersistent (1538x) + 57862: 43, // statsSamplePages (1538x) + 57585: 44, // statsSampleRate (1538x) + 57874: 45, // tableChecksum (1538x) + 57573: 46, // account (1485x) + 57814: 47, // resume (1475x) + 57839: 48, // signed (1475x) + 57845: 49, // snapshot (1474x) + 41: 50, // ')' (1473x) + 57594: 51, // backend (1473x) + 57614: 52, // checkpoint (1473x) + 57631: 53, // concurrency (1473x) + 57637: 54, // csvBackslashEscape (1473x) + 57638: 55, // csvDelimiter (1473x) + 57639: 56, // csvHeader (1473x) + 57640: 57, // csvNotNull (1473x) + 57641: 58, // csvNull (1473x) + 57642: 59, // csvSeparator (1473x) + 57643: 60, // csvTrimLastSeparators (1473x) + 57718: 61, // lastBackup (1473x) + 57764: 62, // onDuplicate (1473x) + 57765: 63, // online (1473x) + 57797: 64, // rateLimit (1473x) + 57828: 65, // sendCredentialsToTiKV (1473x) + 57842: 66, // skipSchemaFiles (1473x) + 57865: 67, // strictFormat (1473x) + 57881: 68, // tikvImporter (1473x) + 57889: 69, // truncate (1470x) + 57751: 70, // no (1469x) + 57859: 71, // start (1467x) + 57608: 72, // cache (1464x) + 57752: 73, // nocache (1463x) + 57646: 74, // cycle (1462x) + 57741: 75, // minValue (1462x) + 57702: 76, // increment (1461x) + 57753: 77, // nocycle (1461x) + 57755: 78, // nomaxvalue (1461x) + 57756: 79, // nominvalue (1461x) + 57811: 80, // restart (1459x) + 57579: 81, // algorithm (1458x) + 57884: 82, // tp (1458x) + 57645: 83, // clustered (1457x) + 57707: 84, // invisible (1457x) + 57757: 85, // nonclustered (1457x) + 58028: 86, // regions (1457x) + 57900: 87, // visible (1457x) + 57918: 88, // constraints (1450x) + 57929: 89, // followerConstraints (1450x) + 57930: 90, // followers (1450x) + 57940: 91, // leaderConstraints (1450x) + 57942: 92, // learnerConstraints (1450x) + 57943: 93, // learners (1450x) + 57953: 94, // primaryRegion (1450x) + 57958: 95, // schedule (1450x) + 57989: 96, // voterConstraints (1450x) + 57990: 97, // voters (1450x) + 57623: 98, // columns (1449x) + 57899: 99, // view (1449x) + 57867: 100, // subpartition (1445x) + 57582: 101, // ascii (1444x) + 57607: 102, // byteType (1444x) + 57774: 103, // partitions (1444x) + 57893: 104, // unicodeSym (1444x) + 57906: 105, // yearType (1444x) + 57650: 106, // day (1443x) + 57680: 107, // fields (1443x) + 57823: 108, // second (1442x) + 57858: 109, // sqlTsiYear (1442x) + 57875: 110, // tables (1442x) + 57697: 111, // hour (1441x) + 57738: 112, // microsecond (1441x) + 57740: 113, // minute (1441x) + 57744: 114, // month (1441x) + 57793: 115, // quarter (1441x) + 57851: 116, // sqlTsiDay (1441x) + 57852: 117, // sqlTsiHour (1441x) + 57853: 118, // sqlTsiMinute (1441x) + 57854: 119, // sqlTsiMonth (1441x) + 57855: 120, // sqlTsiQuarter (1441x) + 57856: 121, // sqlTsiSecond (1441x) + 57857: 122, // sqlTsiWeek (1441x) + 57902: 123, // week (1441x) + 57829: 124, // separator (1440x) + 57863: 125, // status (1440x) + 57730: 126, // maxConnectionsPerHour (1439x) + 57731: 127, // maxQueriesPerHour (1439x) + 57733: 128, // maxUpdatesPerHour (1439x) + 57734: 129, // maxUserConnections (1439x) + 57783: 130, // preceding (1439x) + 57616: 131, // cipher (1438x) + 57700: 132, // importKwd (1438x) + 57712: 133, // issuer (1438x) + 57822: 134, // san (1438x) + 57866: 135, // subject (1438x) + 57723: 136, // local (1437x) + 57841: 137, // skip (1437x) + 57600: 138, // bindings (1436x) + 57652: 139, // definer (1436x) + 57692: 140, // hash (1436x) + 57698: 141, // identified (1436x) + 57726: 142, // logs (1436x) + 57795: 143, // query (1436x) + 57810: 144, // respect (1436x) + 57626: 145, // commit (1435x) + 57644: 146, // current (1435x) + 57664: 147, // enforced (1435x) + 57685: 148, // following (1435x) + 57759: 149, // nowait (1435x) + 57766: 150, // only (1435x) + 57817: 151, // rollback (1435x) + 57897: 152, // value (1435x) + 57597: 153, // begin (1434x) + 57599: 154, // binding (1434x) + 57663: 155, // end (1434x) + 57690: 156, // global (1434x) + 57933: 157, // next_row_id (1434x) + 57781: 158, // policy (1434x) + 57952: 159, // predicate (1434x) + 57877: 160, // temporary (1434x) + 57890: 161, // unbounded (1434x) + 57895: 162, // user (1434x) + 57346: 163, // identifier (1433x) + 57763: 164, // offset (1433x) + 57950: 165, // planCache (1433x) + 57784: 166, // prepare (1433x) + 57816: 167, // role (1433x) + 57894: 168, // unknown (1433x) + 57907: 169, // wait (1433x) + 57606: 170, // btree (1432x) + 57648: 171, // datetimeType (1432x) + 57649: 172, // dateType (1432x) + 57683: 173, // fixed (1432x) + 57711: 174, // isolation (1432x) + 57713: 175, // jsonType (1432x) + 57725: 176, // location (1432x) + 57728: 177, // max_idxnum (1432x) + 57736: 178, // memory (1432x) + 57762: 179, // off (1432x) + 57768: 180, // optional (1432x) + 57777: 181, // per_db (1432x) + 57786: 182, // privileges (1432x) + 57809: 183, // required (1432x) + 57821: 184, // rtree (1432x) + 57956: 185, // running (1432x) + 58011: 186, // sampleRate (1432x) + 57830: 187, // sequence (1432x) + 57833: 188, // session (1432x) + 57844: 189, // slow (1432x) + 57883: 190, // timeType (1432x) + 57896: 191, // validation (1432x) + 57898: 192, // variables (1432x) + 57583: 193, // attributes (1431x) + 57655: 194, // disable (1431x) + 57659: 195, // duplicate (1431x) + 57660: 196, // dynamic (1431x) + 57661: 197, // enable (1431x) + 57668: 198, // errorKwd (1431x) + 57684: 199, // flush (1431x) + 57687: 200, // full (1431x) + 57699: 201, // identSQLErrors (1431x) + 57735: 202, // mb (1431x) + 57742: 203, // mode (1431x) + 57748: 204, // never (1431x) + 57949: 205, // plan (1431x) + 57780: 206, // plugins (1431x) + 57788: 207, // processlist (1431x) + 57799: 208, // recover (1431x) + 57804: 209, // repair (1431x) + 57805: 210, // repeatable (1431x) + 58012: 211, // statistics (1431x) + 57868: 212, // subpartitions (1431x) + 58022: 213, // tidb (1431x) + 57882: 214, // timestampType (1431x) + 57904: 215, // without (1431x) + 57991: 216, // admin (1430x) + 57595: 217, // backup (1430x) + 57601: 218, // binlog (1430x) + 57603: 219, // block (1430x) + 57604: 220, // booleanType (1430x) + 57992: 221, // buckets (1430x) + 57995: 222, // cardinality (1430x) + 57612: 223, // chain (1430x) + 57619: 224, // clientErrorsSummary (1430x) + 57996: 225, // cmSketch (1430x) + 57620: 226, // coalesce (1430x) + 57628: 227, // compact (1430x) + 57629: 228, // compressed (1430x) + 57635: 229, // context (1430x) + 57917: 230, // copyKwd (1430x) + 57998: 231, // correlation (1430x) + 57636: 232, // cpu (1430x) + 57651: 233, // deallocate (1430x) + 58000: 234, // dependency (1430x) + 57654: 235, // directory (1430x) + 57656: 236, // discard (1430x) + 57657: 237, // disk (1430x) + 57658: 238, // do (1430x) + 58002: 239, // drainer (1430x) + 57673: 240, // exchange (1430x) + 57675: 241, // execute (1430x) + 57676: 242, // expansion (1430x) + 57927: 243, // flashback (1430x) + 57689: 244, // general (1430x) + 57693: 245, // help (1430x) + 57694: 246, // histogram (1430x) + 57696: 247, // hosts (1430x) + 57934: 248, // inplace (1430x) + 57706: 249, // instance (1430x) + 57935: 250, // instant (1430x) + 57710: 251, // ipc (1430x) + 58004: 252, // job (1430x) + 58003: 253, // jobs (1430x) + 57715: 254, // labels (1430x) + 57724: 255, // locked (1430x) + 57743: 256, // modify (1430x) + 57749: 257, // next (1430x) + 58005: 258, // nodeID (1430x) + 58006: 259, // nodeState (1430x) + 57761: 260, // nulls (1430x) + 57770: 261, // pageSym (1430x) + 58009: 262, // pump (1430x) + 57792: 263, // purge (1430x) + 57798: 264, // rebuild (1430x) + 57800: 265, // redundant (1430x) + 57801: 266, // reload (1430x) + 57806: 267, // replica (1430x) + 57812: 268, // restore (1430x) + 57818: 269, // routine (1430x) + 57957: 270, // s3 (1430x) + 58010: 271, // samples (1430x) + 57825: 272, // secondaryLoad (1430x) + 57826: 273, // secondaryUnload (1430x) + 57836: 274, // share (1430x) + 57838: 275, // shutdown (1430x) + 57847: 276, // source (1430x) + 58025: 277, // split (1430x) + 58013: 278, // stats (1430x) + 57584: 279, // statsOptions (1430x) + 57964: 280, // stop (1430x) + 57870: 281, // swaps (1430x) + 58023: 282, // tiFlash (1430x) + 57974: 283, // tokudbDefault (1430x) + 57975: 284, // tokudbFast (1430x) + 57976: 285, // tokudbLzma (1430x) + 57977: 286, // tokudbQuickLZ (1430x) + 57979: 287, // tokudbSmall (1430x) + 57978: 288, // tokudbSnappy (1430x) + 57980: 289, // tokudbUncompressed (1430x) + 57981: 290, // tokudbZlib (1430x) + 58024: 291, // topn (1430x) + 57885: 292, // trace (1430x) + 57574: 293, // action (1429x) + 57575: 294, // advise (1429x) + 57577: 295, // against (1429x) + 57578: 296, // ago (1429x) + 57580: 297, // always (1429x) + 57596: 298, // backups (1429x) + 57598: 299, // bernoulli (1429x) + 57602: 300, // bitType (1429x) + 57605: 301, // boolType (1429x) + 57915: 302, // briefType (1429x) + 57993: 303, // builtins (1429x) + 57994: 304, // cancel (1429x) + 57609: 305, // capture (1429x) + 57610: 306, // cascaded (1429x) + 57611: 307, // causal (1429x) + 57617: 308, // cleanup (1429x) + 57618: 309, // client (1429x) + 57621: 310, // collation (1429x) + 57997: 311, // columnStatsUsage (1429x) + 57627: 312, // committed (1429x) + 57624: 313, // config (1429x) + 57633: 314, // consistency (1429x) + 57634: 315, // consistent (1429x) + 57999: 316, // ddl (1429x) + 58001: 317, // depth (1429x) + 57922: 318, // dotType (1429x) + 57923: 319, // dump (1429x) + 57666: 320, // engines (1429x) + 57667: 321, // enum (1429x) + 57671: 322, // events (1429x) + 57672: 323, // evolve (1429x) + 57677: 324, // expire (1429x) + 57925: 325, // exprPushdownBlacklist (1429x) + 57678: 326, // extended (1429x) + 57679: 327, // faultsSym (1429x) + 57686: 328, // format (1429x) + 57688: 329, // function (1429x) + 57691: 330, // grants (1429x) + 58019: 331, // histogramsInFlight (1429x) + 57695: 332, // history (1429x) + 57701: 333, // imports (1429x) + 57703: 334, // incremental (1429x) + 57704: 335, // indexes (1429x) + 57936: 336, // internal (1429x) + 57708: 337, // invoker (1429x) + 57709: 338, // io (1429x) + 57716: 339, // language (1429x) + 57717: 340, // last (1429x) + 57720: 341, // less (1429x) + 57721: 342, // level (1429x) + 57722: 343, // list (1429x) + 57727: 344, // master (1429x) + 57729: 345, // max_minutes (1429x) + 57737: 346, // merge (1429x) + 57746: 347, // national (1429x) + 57747: 348, // ncharType (1429x) + 57750: 349, // nextval (1429x) + 57758: 350, // none (1429x) + 57760: 351, // nvarcharType (1429x) + 57767: 352, // open (1429x) + 58007: 353, // optimistic (1429x) + 57947: 354, // optRuleBlacklist (1429x) + 57771: 355, // parser (1429x) + 57772: 356, // partial (1429x) + 57773: 357, // partitioning (1429x) + 57778: 358, // per_table (1429x) + 57776: 359, // percent (1429x) + 58008: 360, // pessimistic (1429x) + 57785: 361, // preserve (1429x) + 57789: 362, // profile (1429x) + 57790: 363, // profiles (1429x) + 57794: 364, // queries (1429x) + 57954: 365, // recent (1429x) + 58029: 366, // region (1429x) + 57955: 367, // replayer (1429x) + 58027: 368, // reset (1429x) + 57813: 369, // restores (1429x) + 57827: 370, // security (1429x) + 57832: 371, // serializable (1429x) + 57840: 372, // simple (1429x) + 57843: 373, // slave (1429x) + 58017: 374, // statsHealthy (1429x) + 58015: 375, // statsHistograms (1429x) + 58014: 376, // statsMeta (1429x) + 57965: 377, // strict (1429x) + 57871: 378, // switchesSym (1429x) + 57872: 379, // system (1429x) + 57873: 380, // systemTime (1429x) + 57970: 381, // target (1429x) + 58021: 382, // telemetryID (1429x) + 57878: 383, // temptable (1429x) + 57879: 384, // textType (1429x) + 57880: 385, // than (1429x) + 57973: 386, // tls (1429x) + 57982: 387, // top (1429x) + 57886: 388, // traditional (1429x) + 57887: 389, // transaction (1429x) + 57888: 390, // triggers (1429x) + 57891: 391, // uncommitted (1429x) + 57892: 392, // undefined (1429x) + 57987: 393, // verboseType (1429x) + 57901: 394, // warnings (1429x) + 58026: 395, // width (1429x) + 57905: 396, // x509 (1429x) + 57908: 397, // addDate (1428x) + 57581: 398, // any (1428x) + 57909: 399, // approxCountDistinct (1428x) + 57910: 400, // approxPercentile (1428x) + 57592: 401, // avg (1428x) + 57911: 402, // bitAnd (1428x) + 57912: 403, // bitOr (1428x) + 57913: 404, // bitXor (1428x) + 57914: 405, // bound (1428x) + 57916: 406, // cast (1428x) + 57919: 407, // curTime (1428x) + 57920: 408, // dateAdd (1428x) + 57921: 409, // dateSub (1428x) + 57669: 410, // escape (1428x) + 57670: 411, // event (1428x) + 57924: 412, // exact (1428x) + 57674: 413, // exclusive (1428x) + 57926: 414, // extract (1428x) + 57681: 415, // file (1428x) + 57928: 416, // follower (1428x) + 57931: 417, // getFormat (1428x) + 57932: 418, // groupConcat (1428x) + 57937: 419, // jsonArrayagg (1428x) + 57938: 420, // jsonObjectAgg (1428x) + 57719: 421, // lastval (1428x) + 57939: 422, // leader (1428x) + 57941: 423, // learner (1428x) + 57945: 424, // max (1428x) + 57944: 425, // min (1428x) + 57745: 426, // names (1428x) + 57946: 427, // now (1428x) + 57951: 428, // position (1428x) + 57787: 429, // process (1428x) + 57791: 430, // proxy (1428x) + 57796: 431, // quick (1428x) + 57807: 432, // replicas (1428x) + 57808: 433, // replication (1428x) + 57815: 434, // reverse (1428x) + 57819: 435, // rowCount (1428x) + 57834: 436, // setval (1428x) + 57837: 437, // shared (1428x) + 57846: 438, // some (1428x) + 57848: 439, // sqlBufferResult (1428x) + 57849: 440, // sqlCache (1428x) + 57850: 441, // sqlNoCache (1428x) + 57959: 442, // staleness (1428x) + 57960: 443, // std (1428x) + 57961: 444, // stddev (1428x) + 57962: 445, // stddevPop (1428x) + 57963: 446, // stddevSamp (1428x) + 57966: 447, // strong (1428x) + 57967: 448, // subDate (1428x) + 57969: 449, // substring (1428x) + 57968: 450, // sum (1428x) + 57869: 451, // super (1428x) + 58020: 452, // telemetry (1428x) + 57971: 453, // timestampAdd (1428x) + 57972: 454, // timestampDiff (1428x) + 57983: 455, // trim (1428x) + 57984: 456, // variance (1428x) + 57985: 457, // varPop (1428x) + 57986: 458, // varSamp (1428x) + 57988: 459, // voter (1428x) + 57903: 460, // weightString (1428x) + 57488: 461, // on (1362x) 40: 462, // '(' (1276x) - 57568: 463, // with (1176x) - 57349: 464, // stringLit (1165x) - 58077: 465, // not2 (1159x) - 57481: 466, // not (1104x) - 57364: 467, // as (1073x) - 57398: 468, // defaultKwd (1065x) - 57547: 469, // union (1041x) - 57553: 470, // using (1034x) - 57461: 471, // left (1021x) - 57515: 472, // right (1021x) - 57379: 473, // collate (1015x) - 45: 474, // '-' (990x) - 43: 475, // '+' (989x) - 57480: 476, // mod (970x) - 57415: 477, // except (934x) - 57441: 478, // intersect (933x) - 57435: 479, // ignore (932x) + 57568: 463, // with (1178x) + 57349: 464, // stringLit (1167x) + 58075: 465, // not2 (1161x) + 57481: 466, // not (1106x) + 57364: 467, // as (1075x) + 57398: 468, // defaultKwd (1070x) + 57547: 469, // union (1043x) + 57553: 470, // using (1036x) + 57461: 471, // left (1023x) + 57515: 472, // right (1023x) + 57379: 473, // collate (1022x) + 45: 474, // '-' (992x) + 43: 475, // '+' (991x) + 57480: 476, // mod (972x) + 57415: 477, // except (936x) + 57441: 478, // intersect (935x) + 57435: 479, // ignore (934x) 57496: 480, // partition (928x) 57485: 481, // null (915x) - 57420: 482, // forKwd (907x) - 57463: 483, // limit (907x) - 57443: 484, // into (904x) - 57469: 485, // lock (900x) - 57423: 486, // from (891x) - 58066: 487, // eq (890x) - 57417: 488, // fetch (890x) - 57565: 489, // where (889x) - 57493: 490, // order (886x) + 57420: 482, // forKwd (909x) + 57463: 483, // limit (909x) + 57443: 484, // into (906x) + 57469: 485, // lock (902x) + 57423: 486, // from (893x) + 58064: 487, // eq (892x) + 57417: 488, // fetch (892x) + 57565: 489, // where (891x) + 57493: 490, // order (888x) 57557: 491, // values (886x) - 57421: 492, // force (882x) - 57363: 493, // and (871x) - 57377: 494, // charType (867x) - 57511: 495, // replace (859x) - 58061: 496, // intLit (856x) - 57492: 497, // or (848x) - 57354: 498, // andand (847x) - 57779: 499, // pipesAsOr (847x) - 57569: 500, // xor (847x) - 57522: 501, // set (845x) - 57427: 502, // group (820x) - 57533: 503, // straightJoin (816x) - 57567: 504, // window (808x) - 57429: 505, // having (806x) - 57453: 506, // join (804x) - 57572: 507, // natural (794x) - 57384: 508, // cross (793x) - 57439: 509, // inner (793x) - 57462: 510, // like (792x) - 125: 511, // '}' (790x) - 42: 512, // '*' (785x) - 57518: 513, // rows (778x) - 57552: 514, // use (774x) - 57535: 515, // tableSample (768x) - 57501: 516, // rangeKwd (767x) - 57428: 517, // groups (766x) - 57402: 518, // desc (765x) - 57365: 519, // asc (763x) - 57393: 520, // dayHour (761x) - 57394: 521, // dayMicrosecond (761x) - 57395: 522, // dayMinute (761x) - 57396: 523, // daySecond (761x) - 57431: 524, // hourMicrosecond (761x) - 57432: 525, // hourMinute (761x) - 57433: 526, // hourSecond (761x) - 57478: 527, // minuteMicrosecond (761x) - 57479: 528, // minuteSecond (761x) - 57520: 529, // secondMicrosecond (761x) - 57570: 530, // yearMonth (761x) - 57564: 531, // when (760x) - 57368: 532, // binaryType (758x) - 57436: 533, // in (758x) - 57410: 534, // elseKwd (757x) - 57538: 535, // then (754x) - 60: 536, // '<' (747x) - 62: 537, // '>' (747x) - 58067: 538, // ge (747x) - 57445: 539, // is (747x) - 58068: 540, // le (747x) - 58072: 541, // neq (747x) - 58073: 542, // neqSynonym (747x) - 58074: 543, // nulleq (747x) - 57366: 544, // between (745x) - 47: 545, // '/' (744x) - 37: 546, // '%' (743x) - 38: 547, // '&' (743x) - 94: 548, // '^' (743x) - 124: 549, // '|' (743x) - 57406: 550, // div (743x) - 58071: 551, // lsh (743x) - 58076: 552, // rsh (743x) - 57507: 553, // regexpKwd (737x) - 57516: 554, // rlike (737x) + 57421: 492, // force (884x) + 57522: 493, // set (876x) + 57363: 494, // and (873x) + 57377: 495, // charType (872x) + 57511: 496, // replace (859x) + 58059: 497, // intLit (857x) + 57492: 498, // or (850x) + 57354: 499, // andand (849x) + 57779: 500, // pipesAsOr (849x) + 57569: 501, // xor (849x) + 57427: 502, // group (822x) + 57533: 503, // straightJoin (818x) + 57567: 504, // window (810x) + 57429: 505, // having (808x) + 57453: 506, // join (806x) + 57572: 507, // natural (796x) + 57384: 508, // cross (795x) + 57439: 509, // inner (795x) + 57462: 510, // like (794x) + 125: 511, // '}' (792x) + 42: 512, // '*' (787x) + 57518: 513, // rows (780x) + 57552: 514, // use (776x) + 57535: 515, // tableSample (770x) + 57501: 516, // rangeKwd (769x) + 57428: 517, // groups (768x) + 57402: 518, // desc (767x) + 57365: 519, // asc (765x) + 57393: 520, // dayHour (763x) + 57394: 521, // dayMicrosecond (763x) + 57395: 522, // dayMinute (763x) + 57396: 523, // daySecond (763x) + 57431: 524, // hourMicrosecond (763x) + 57432: 525, // hourMinute (763x) + 57433: 526, // hourSecond (763x) + 57478: 527, // minuteMicrosecond (763x) + 57479: 528, // minuteSecond (763x) + 57520: 529, // secondMicrosecond (763x) + 57570: 530, // yearMonth (763x) + 57564: 531, // when (762x) + 57436: 532, // in (760x) + 57410: 533, // elseKwd (759x) + 57368: 534, // binaryType (758x) + 57538: 535, // then (756x) + 60: 536, // '<' (749x) + 62: 537, // '>' (749x) + 58065: 538, // ge (749x) + 57445: 539, // is (749x) + 58066: 540, // le (749x) + 58070: 541, // neq (749x) + 58071: 542, // neqSynonym (749x) + 58072: 543, // nulleq (749x) + 57366: 544, // between (747x) + 47: 545, // '/' (746x) + 37: 546, // '%' (745x) + 38: 547, // '&' (745x) + 94: 548, // '^' (745x) + 124: 549, // '|' (745x) + 57406: 550, // div (745x) + 58069: 551, // lsh (745x) + 58074: 552, // rsh (745x) + 57507: 553, // regexpKwd (739x) + 57516: 554, // rlike (739x) 57434: 555, // ifKwd (733x) 57446: 556, // insert (715x) 57350: 557, // singleAtIdentifier (715x) @@ -1383,808 +1381,806 @@ var ( 57534: 559, // tableKwd (710x) 57416: 560, // falseKwd (709x) 57545: 561, // trueKwd (709x) - 58060: 562, // decLit (703x) - 58059: 563, // floatLit (703x) + 58058: 562, // decLit (703x) + 58057: 563, // floatLit (703x) 57517: 564, // row (702x) - 58062: 565, // hexLit (701x) + 58060: 565, // hexLit (701x) 57454: 566, // key (701x) - 58075: 567, // paramMarker (701x) + 58073: 567, // paramMarker (701x) 123: 568, // '{' (699x) - 58063: 569, // bitLit (699x) + 58061: 569, // bitLit (699x) 57442: 570, // interval (698x) - 57355: 571, // pipes (695x) + 57355: 571, // pipes (697x) 57391: 572, // database (694x) 57413: 573, // exists (694x) 57378: 574, // check (691x) 57382: 575, // convert (691x) 57499: 576, // primary (691x) 57351: 577, // doubleAtIdentifier (690x) - 58046: 578, // builtinNow (689x) + 58045: 578, // builtinNow (689x) 57388: 579, // currentTs (689x) 57467: 580, // localTime (689x) 57468: 581, // localTs (689x) 57348: 582, // underscoreCS (689x) 33: 583, // '!' (687x) 126: 584, // '~' (687x) - 58030: 585, // builtinAddDate (687x) - 58036: 586, // builtinApproxCountDistinct (687x) - 58037: 587, // builtinApproxPercentile (687x) - 58031: 588, // builtinBitAnd (687x) - 58032: 589, // builtinBitOr (687x) - 58033: 590, // builtinBitXor (687x) - 58034: 591, // builtinCast (687x) - 58035: 592, // builtinCount (687x) - 58038: 593, // builtinCurDate (687x) - 58039: 594, // builtinCurTime (687x) - 58040: 595, // builtinDateAdd (687x) - 58041: 596, // builtinDateSub (687x) - 58042: 597, // builtinExtract (687x) - 58043: 598, // builtinGroupConcat (687x) - 58044: 599, // builtinMax (687x) - 58045: 600, // builtinMin (687x) - 58047: 601, // builtinPosition (687x) - 58052: 602, // builtinStddevPop (687x) - 58053: 603, // builtinStddevSamp (687x) - 58048: 604, // builtinSubDate (687x) - 58049: 605, // builtinSubstring (687x) - 58050: 606, // builtinSum (687x) - 58051: 607, // builtinSysDate (687x) - 58054: 608, // builtinTranslate (687x) - 58055: 609, // builtinTrim (687x) - 58056: 610, // builtinUser (687x) - 58057: 611, // builtinVarPop (687x) - 58058: 612, // builtinVarSamp (687x) - 57374: 613, // caseKwd (687x) - 57385: 614, // cumeDist (687x) - 57386: 615, // currentDate (687x) - 57390: 616, // currentRole (687x) - 57387: 617, // currentTime (687x) - 57401: 618, // denseRank (687x) - 57418: 619, // firstValue (687x) - 57457: 620, // lag (687x) - 57458: 621, // lastValue (687x) - 57459: 622, // lead (687x) - 57483: 623, // nthValue (687x) - 57484: 624, // ntile (687x) - 57497: 625, // percentRank (687x) - 57502: 626, // rank (687x) - 57510: 627, // repeat (687x) - 57519: 628, // rowNumber (687x) - 57554: 629, // utcDate (687x) - 57556: 630, // utcTime (687x) - 57555: 631, // utcTimestamp (687x) - 57546: 632, // unique (684x) - 57381: 633, // constraint (682x) - 57506: 634, // references (679x) - 57425: 635, // generated (675x) - 57521: 636, // selectKwd (667x) - 57376: 637, // character (641x) - 57473: 638, // match (637x) - 57437: 639, // index (634x) - 57542: 640, // to (556x) - 57360: 641, // all (543x) - 46: 642, // '.' (534x) - 57362: 643, // analyze (518x) - 57550: 644, // update (507x) - 58069: 645, // jss (502x) - 58070: 646, // juss (502x) - 57474: 647, // maxValue (500x) - 57464: 648, // lines (493x) - 57371: 649, // by (490x) - 58065: 650, // assignmentEq (488x) - 57512: 651, // require (485x) - 57361: 652, // alter (484x) - 58322: 653, // Identifier (483x) - 58397: 654, // NotKeywordToken (483x) - 58618: 655, // TiDBKeyword (483x) - 58628: 656, // UnReservedKeyword (483x) - 64: 657, // '@' (480x) - 57526: 658, // sql (477x) - 57408: 659, // drop (474x) - 57373: 660, // cascade (473x) - 57503: 661, // read (473x) - 57513: 662, // restrict (473x) - 57347: 663, // asof (471x) - 57383: 664, // create (469x) - 57422: 665, // foreign (469x) - 57424: 666, // fulltext (469x) - 57560: 667, // varcharacter (467x) - 57559: 668, // varcharType (467x) - 57375: 669, // change (466x) - 57397: 670, // decimalType (466x) - 57407: 671, // doubleType (466x) - 57419: 672, // floatType (466x) - 57440: 673, // integerType (466x) - 57447: 674, // intType (466x) - 57504: 675, // realType (466x) - 57509: 676, // rename (466x) - 57566: 677, // write (466x) - 57561: 678, // varbinaryType (465x) - 57359: 679, // add (464x) - 57367: 680, // bigIntType (464x) - 57369: 681, // blobType (464x) - 57448: 682, // int1Type (464x) - 57449: 683, // int2Type (464x) - 57450: 684, // int3Type (464x) - 57451: 685, // int4Type (464x) - 57452: 686, // int8Type (464x) - 57558: 687, // long (464x) - 57470: 688, // longblobType (464x) - 57471: 689, // longtextType (464x) - 57475: 690, // mediumblobType (464x) - 57476: 691, // mediumIntType (464x) - 57477: 692, // mediumtextType (464x) - 57486: 693, // numericType (464x) - 57489: 694, // optimize (464x) - 57524: 695, // smallIntType (464x) - 57539: 696, // tinyblobType (464x) - 57540: 697, // tinyIntType (464x) - 57541: 698, // tinytextType (464x) - 58583: 699, // SubSelect (209x) - 58637: 700, // UserVariable (171x) - 58558: 701, // SimpleIdent (170x) - 58374: 702, // Literal (168x) - 58573: 703, // StringLiteral (168x) - 58395: 704, // NextValueForSequence (167x) - 58299: 705, // FunctionCallGeneric (166x) - 58300: 706, // FunctionCallKeyword (166x) - 58301: 707, // FunctionCallNonKeyword (166x) - 58302: 708, // FunctionNameConflict (166x) - 58303: 709, // FunctionNameDateArith (166x) - 58304: 710, // FunctionNameDateArithMultiForms (166x) - 58305: 711, // FunctionNameDatetimePrecision (166x) - 58306: 712, // FunctionNameOptionalBraces (166x) - 58307: 713, // FunctionNameSequence (166x) - 58557: 714, // SimpleExpr (166x) - 58584: 715, // SumExpr (166x) - 58586: 716, // SystemVariable (166x) - 58648: 717, // Variable (166x) - 58671: 718, // WindowFuncCall (166x) - 58151: 719, // BitExpr (153x) - 58467: 720, // PredicateExpr (130x) - 58154: 721, // BoolPri (127x) - 58266: 722, // Expression (127x) - 58686: 723, // logAnd (96x) - 58687: 724, // logOr (96x) - 58393: 725, // NUM (96x) - 58256: 726, // EqOpt (75x) - 58596: 727, // TableName (75x) - 58574: 728, // StringName (56x) - 57549: 729, // unsigned (47x) - 57495: 730, // over (45x) - 57571: 731, // zerofill (45x) - 57400: 732, // deleteKwd (41x) - 58176: 733, // ColumnName (40x) - 58365: 734, // LengthNum (40x) - 57404: 735, // distinct (36x) - 57405: 736, // distinctRow (36x) - 58676: 737, // WindowingClause (35x) - 57399: 738, // delayed (33x) - 57430: 739, // highPriority (33x) - 57472: 740, // lowPriority (33x) - 58513: 741, // SelectStmt (30x) - 58514: 742, // SelectStmtBasic (30x) - 58516: 743, // SelectStmtFromDualTable (30x) - 58517: 744, // SelectStmtFromTable (30x) - 58533: 745, // SetOprClause (30x) - 58534: 746, // SetOprClauseList (29x) - 58537: 747, // SetOprStmtWithLimitOrderBy (29x) - 58538: 748, // SetOprStmtWoutLimitOrderBy (29x) - 57353: 749, // hintComment (27x) - 58277: 750, // FieldLen (26x) - 58354: 751, // Int64Num (26x) - 58526: 752, // SelectStmtWithClause (26x) - 58536: 753, // SetOprStmt (26x) - 58677: 754, // WithClause (26x) - 58434: 755, // OptWindowingClause (24x) - 58439: 756, // OrderBy (23x) - 58520: 757, // SelectStmtLimit (23x) - 57527: 758, // sqlBigResult (23x) - 57528: 759, // sqlCalcFoundRows (23x) - 57529: 760, // sqlSmallResult (23x) - 58164: 761, // CharsetKw (20x) - 58639: 762, // Username (20x) - 58631: 763, // UpdateStmtNoWith (18x) - 58232: 764, // DeleteWithoutUsingStmt (17x) - 58267: 765, // ExpressionList (17x) - 58462: 766, // PlacementPolicyOption (17x) - 58323: 767, // IfExists (16x) - 58351: 768, // InsertIntoStmt (16x) - 58488: 769, // ReplaceIntoStmt (16x) - 57537: 770, // terminated (16x) - 58630: 771, // UpdateStmt (16x) - 58234: 772, // DistinctKwd (15x) - 58324: 773, // IfNotExists (15x) - 58419: 774, // OptFieldLen (15x) - 58235: 775, // DistinctOpt (14x) - 57411: 776, // enclosed (14x) - 58450: 777, // PartitionNameList (14x) - 58661: 778, // WhereClause (14x) - 58662: 779, // WhereClauseOptional (14x) - 58227: 780, // DefaultKwdOpt (13x) - 58231: 781, // DeleteWithUsingStmt (13x) - 57412: 782, // escaped (13x) - 57491: 783, // optionally (13x) - 58597: 784, // TableNameList (13x) - 58230: 785, // DeleteFromStmt (12x) - 58265: 786, // ExprOrDefault (12x) - 58359: 787, // JoinTable (12x) - 58413: 788, // OptBinary (12x) - 58504: 789, // RolenameComposed (12x) - 58593: 790, // TableFactor (12x) - 58606: 791, // TableRef (12x) - 58126: 792, // AnalyzeOptionListOpt (11x) - 58294: 793, // FromOrIn (11x) - 58620: 794, // TimestampUnit (11x) - 58165: 795, // CharsetName (10x) - 58177: 796, // ColumnNameList (10x) - 57466: 797, // load (10x) - 58398: 798, // NotSym (10x) - 58440: 799, // OrderByOptional (10x) - 58442: 800, // PartDefOption (10x) - 58556: 801, // SignedNum (10x) - 58157: 802, // BuggyDefaultFalseDistinctOpt (9x) - 58217: 803, // DBName (9x) - 58226: 804, // DefaultFalseDistinctOpt (9x) - 58360: 805, // JoinType (9x) - 57482: 806, // noWriteToBinLog (9x) - 58403: 807, // NumLiteral (9x) - 58503: 808, // Rolename (9x) - 58498: 809, // RoleNameString (9x) - 58122: 810, // AlterTableStmt (8x) - 58216: 811, // CrossOpt (8x) - 58257: 812, // EqOrAssignmentEq (8x) - 58268: 813, // ExpressionListOpt (8x) - 58345: 814, // IndexPartSpecification (8x) - 58361: 815, // KeyOrIndex (8x) - 58521: 816, // SelectStmtLimitOpt (8x) - 58619: 817, // TimeUnit (8x) - 58651: 818, // VariableName (8x) - 58108: 819, // AllOrPartitionNameList (7x) - 58200: 820, // ConstraintKeywordOpt (7x) - 58283: 821, // FieldsOrColumns (7x) - 58292: 822, // ForceOpt (7x) - 58346: 823, // IndexPartSpecificationList (7x) - 58396: 824, // NoWriteToBinLogAliasOpt (7x) - 58471: 825, // Priority (7x) - 58508: 826, // RowFormat (7x) - 58511: 827, // RowValue (7x) - 58531: 828, // SetExpr (7x) - 58542: 829, // ShowDatabaseNameOpt (7x) - 58603: 830, // TableOption (7x) - 57562: 831, // varying (7x) - 58147: 832, // BeginTransactionStmt (6x) - 57380: 833, // column (6x) - 58171: 834, // ColumnDef (6x) - 58190: 835, // CommitStmt (6x) - 58219: 836, // DatabaseOption (6x) - 58222: 837, // DatabaseSym (6x) - 58259: 838, // EscapedTableRef (6x) - 58264: 839, // ExplainableStmt (6x) - 58281: 840, // FieldTerminator (6x) - 57426: 841, // grant (6x) - 58328: 842, // IgnoreOptional (6x) - 58337: 843, // IndexInvisible (6x) - 58342: 844, // IndexNameList (6x) - 58348: 845, // IndexType (6x) - 58378: 846, // LoadDataStmt (6x) - 58451: 847, // PartitionNameListOpt (6x) - 57508: 848, // release (6x) - 58505: 849, // RolenameList (6x) - 58507: 850, // RollbackStmt (6x) - 58541: 851, // SetStmt (6x) - 57523: 852, // show (6x) - 58601: 853, // TableOptimizerHints (6x) - 58640: 854, // UsernameList (6x) - 58678: 855, // WithClustered (6x) - 58106: 856, // AlgorithmClause (5x) - 58158: 857, // ByItem (5x) - 58170: 858, // CollationName (5x) - 58174: 859, // ColumnKeywordOpt (5x) - 58233: 860, // DirectPlacementOption (5x) - 58279: 861, // FieldOpt (5x) - 58280: 862, // FieldOpts (5x) - 58320: 863, // IdentList (5x) - 58340: 864, // IndexName (5x) - 58343: 865, // IndexOption (5x) - 58344: 866, // IndexOptionList (5x) - 57438: 867, // infile (5x) - 58370: 868, // LimitOption (5x) - 58382: 869, // LockClause (5x) - 58415: 870, // OptCharsetWithOptBinary (5x) - 58426: 871, // OptNullTreatment (5x) - 58465: 872, // PolicyName (5x) - 58472: 873, // PriorityOpt (5x) - 58512: 874, // SelectLockOpt (5x) - 58519: 875, // SelectStmtIntoOption (5x) - 58607: 876, // TableRefs (5x) - 58633: 877, // UserSpec (5x) - 58132: 878, // Assignment (4x) - 58138: 879, // AuthString (4x) - 58149: 880, // BindableStmt (4x) - 58139: 881, // BRIEBooleanOptionName (4x) - 58140: 882, // BRIEIntegerOptionName (4x) - 58141: 883, // BRIEKeywordOptionName (4x) - 58142: 884, // BRIEOption (4x) - 58143: 885, // BRIEOptions (4x) - 58145: 886, // BRIEStringOptionName (4x) - 58159: 887, // ByList (4x) - 58163: 888, // Char (4x) - 58194: 889, // ConfigItemName (4x) - 58198: 890, // Constraint (4x) - 58288: 891, // FloatOpt (4x) - 58349: 892, // IndexTypeName (4x) - 57490: 893, // option (4x) - 58431: 894, // OptWild (4x) - 57494: 895, // outer (4x) - 58466: 896, // Precision (4x) - 58480: 897, // ReferDef (4x) - 58494: 898, // RestrictOrCascadeOpt (4x) - 58510: 899, // RowStmt (4x) - 58527: 900, // SequenceOption (4x) - 57532: 901, // statsExtended (4x) - 58588: 902, // TableAsName (4x) - 58589: 903, // TableAsNameOpt (4x) - 58600: 904, // TableNameOptWild (4x) - 58602: 905, // TableOptimizerHintsOpt (4x) - 58604: 906, // TableOptionList (4x) - 58622: 907, // TraceableStmt (4x) - 58623: 908, // TransactionChar (4x) - 58634: 909, // UserSpecList (4x) - 58672: 910, // WindowName (4x) - 58129: 911, // AsOfClause (3x) - 58133: 912, // AssignmentList (3x) - 58135: 913, // AttributesOpt (3x) - 58155: 914, // Boolean (3x) - 58183: 915, // ColumnOption (3x) - 58186: 916, // ColumnPosition (3x) - 58191: 917, // CommonTableExpr (3x) - 58212: 918, // CreateTableStmt (3x) - 58220: 919, // DatabaseOptionList (3x) - 58228: 920, // DefaultTrueDistinctOpt (3x) - 58253: 921, // EnforcedOrNot (3x) - 57414: 922, // explain (3x) - 58270: 923, // ExtendedPriv (3x) - 58308: 924, // GeneratedAlways (3x) - 58310: 925, // GlobalScope (3x) - 58314: 926, // GroupByClause (3x) - 58332: 927, // IndexHint (3x) - 58336: 928, // IndexHintType (3x) - 58341: 929, // IndexNameAndTypeOpt (3x) - 57455: 930, // keys (3x) - 58372: 931, // Lines (3x) - 58390: 932, // MaxValueOrExpression (3x) - 58427: 933, // OptOrder (3x) - 58430: 934, // OptTemporary (3x) - 58443: 935, // PartDefOptionList (3x) - 58445: 936, // PartitionDefinition (3x) - 58454: 937, // PasswordExpire (3x) - 58456: 938, // PasswordOrLockOption (3x) - 58464: 939, // PluginNameList (3x) - 58470: 940, // PrimaryOpt (3x) - 58473: 941, // PrivElem (3x) - 58475: 942, // PrivType (3x) - 57500: 943, // procedure (3x) - 58489: 944, // RequireClause (3x) - 58490: 945, // RequireClauseOpt (3x) - 58492: 946, // RequireListElement (3x) - 58506: 947, // RolenameWithoutIdent (3x) - 58499: 948, // RoleOrPrivElem (3x) - 58518: 949, // SelectStmtGroup (3x) - 58535: 950, // SetOprOpt (3x) - 58587: 951, // TableAliasRefList (3x) - 58590: 952, // TableElement (3x) - 58599: 953, // TableNameListOpt2 (3x) - 58615: 954, // TextString (3x) - 58624: 955, // TransactionChars (3x) - 57544: 956, // trigger (3x) - 57548: 957, // unlock (3x) - 57551: 958, // usage (3x) - 58644: 959, // ValuesList (3x) - 58646: 960, // ValuesStmtList (3x) - 58642: 961, // ValueSym (3x) - 58649: 962, // VariableAssignment (3x) - 58669: 963, // WindowFrameStart (3x) - 58105: 964, // AdminStmt (2x) - 58107: 965, // AllColumnsOrPredicateColumnsOpt (2x) - 58109: 966, // AlterDatabaseStmt (2x) - 58110: 967, // AlterImportStmt (2x) - 58111: 968, // AlterInstanceStmt (2x) - 58112: 969, // AlterOrderItem (2x) - 58114: 970, // AlterPolicyStmt (2x) - 58115: 971, // AlterSequenceOption (2x) - 58117: 972, // AlterSequenceStmt (2x) - 58119: 973, // AlterTableSpec (2x) - 58123: 974, // AlterUserStmt (2x) - 58124: 975, // AnalyzeOption (2x) - 58127: 976, // AnalyzeTableStmt (2x) - 58150: 977, // BinlogStmt (2x) - 58144: 978, // BRIEStmt (2x) - 58146: 979, // BRIETables (2x) - 57372: 980, // call (2x) - 58160: 981, // CallStmt (2x) - 58161: 982, // CastType (2x) - 58162: 983, // ChangeStmt (2x) - 58168: 984, // CheckConstraintKeyword (2x) - 58178: 985, // ColumnNameListOpt (2x) - 58181: 986, // ColumnNameOrUserVariable (2x) - 58184: 987, // ColumnOptionList (2x) - 58185: 988, // ColumnOptionListOpt (2x) - 58187: 989, // ColumnSetValue (2x) - 58193: 990, // CompletionTypeWithinTransaction (2x) - 58195: 991, // ConnectionOption (2x) - 58197: 992, // ConnectionOptions (2x) - 58201: 993, // CreateBindingStmt (2x) - 58202: 994, // CreateDatabaseStmt (2x) - 58203: 995, // CreateImportStmt (2x) - 58204: 996, // CreateIndexStmt (2x) - 58205: 997, // CreatePolicyStmt (2x) - 58206: 998, // CreateRoleStmt (2x) - 58208: 999, // CreateSequenceStmt (2x) - 58209: 1000, // CreateStatisticsStmt (2x) - 58210: 1001, // CreateTableOptionListOpt (2x) - 58213: 1002, // CreateUserStmt (2x) - 58215: 1003, // CreateViewStmt (2x) - 57392: 1004, // databases (2x) - 58224: 1005, // DeallocateStmt (2x) - 58225: 1006, // DeallocateSym (2x) - 57403: 1007, // describe (2x) - 58236: 1008, // DoStmt (2x) - 58237: 1009, // DropBindingStmt (2x) - 58238: 1010, // DropDatabaseStmt (2x) - 58239: 1011, // DropImportStmt (2x) - 58240: 1012, // DropIndexStmt (2x) - 58241: 1013, // DropPolicyStmt (2x) - 58242: 1014, // DropRoleStmt (2x) - 58243: 1015, // DropSequenceStmt (2x) - 58244: 1016, // DropStatisticsStmt (2x) - 58245: 1017, // DropStatsStmt (2x) - 58246: 1018, // DropTableStmt (2x) - 58247: 1019, // DropUserStmt (2x) - 58248: 1020, // DropViewStmt (2x) - 58249: 1021, // DuplicateOpt (2x) - 58251: 1022, // EmptyStmt (2x) - 58252: 1023, // EncryptionOpt (2x) - 58254: 1024, // EnforcedOrNotOpt (2x) - 58258: 1025, // ErrorHandling (2x) - 58260: 1026, // ExecuteStmt (2x) - 58262: 1027, // ExplainStmt (2x) - 58263: 1028, // ExplainSym (2x) - 58272: 1029, // Field (2x) - 58275: 1030, // FieldItem (2x) - 58282: 1031, // Fields (2x) - 58286: 1032, // FlashbackTableStmt (2x) - 58291: 1033, // FlushStmt (2x) - 58297: 1034, // FuncDatetimePrecList (2x) - 58298: 1035, // FuncDatetimePrecListOpt (2x) - 58311: 1036, // GrantProxyStmt (2x) - 58312: 1037, // GrantRoleStmt (2x) - 58313: 1038, // GrantStmt (2x) - 58315: 1039, // HandleRange (2x) - 58317: 1040, // HashString (2x) - 58319: 1041, // HelpStmt (2x) - 58331: 1042, // IndexAdviseStmt (2x) - 58333: 1043, // IndexHintList (2x) - 58334: 1044, // IndexHintListOpt (2x) - 58339: 1045, // IndexLockAndAlgorithmOpt (2x) - 58352: 1046, // InsertValues (2x) - 58356: 1047, // IntoOpt (2x) - 58362: 1048, // KeyOrIndexOpt (2x) - 57456: 1049, // kill (2x) - 58363: 1050, // KillOrKillTiDB (2x) - 58364: 1051, // KillStmt (2x) - 58369: 1052, // LimitClause (2x) - 57465: 1053, // linear (2x) - 58371: 1054, // LinearOpt (2x) - 58375: 1055, // LoadDataSetItem (2x) - 58379: 1056, // LoadStatsStmt (2x) - 58380: 1057, // LocalOpt (2x) - 58383: 1058, // LockTablesStmt (2x) - 58391: 1059, // MaxValueOrExpressionList (2x) - 58399: 1060, // NowSym (2x) - 58400: 1061, // NowSymFunc (2x) - 58401: 1062, // NowSymOptionFraction (2x) - 58402: 1063, // NumList (2x) - 58405: 1064, // ObjectType (2x) - 57487: 1065, // of (2x) - 58406: 1066, // OfTablesOpt (2x) - 58407: 1067, // OnCommitOpt (2x) - 58408: 1068, // OnDelete (2x) - 58411: 1069, // OnUpdate (2x) - 58416: 1070, // OptCollate (2x) - 58421: 1071, // OptFull (2x) - 58423: 1072, // OptInteger (2x) - 58436: 1073, // OptionalBraces (2x) - 58435: 1074, // OptionLevel (2x) - 58425: 1075, // OptLeadLagInfo (2x) - 58424: 1076, // OptLLDefault (2x) - 58441: 1077, // OuterOpt (2x) - 58446: 1078, // PartitionDefinitionList (2x) - 58447: 1079, // PartitionDefinitionListOpt (2x) - 58453: 1080, // PartitionOpt (2x) - 58455: 1081, // PasswordOpt (2x) - 58457: 1082, // PasswordOrLockOptionList (2x) - 58458: 1083, // PasswordOrLockOptions (2x) - 58461: 1084, // PlacementOptionList (2x) - 58463: 1085, // PlanReplayerStmt (2x) - 58469: 1086, // PreparedStmt (2x) - 58474: 1087, // PrivLevel (2x) - 58477: 1088, // PurgeImportStmt (2x) - 58478: 1089, // QuickOptional (2x) - 58479: 1090, // RecoverTableStmt (2x) - 58481: 1091, // ReferOpt (2x) - 58483: 1092, // RegexpSym (2x) - 58484: 1093, // RenameTableStmt (2x) - 58485: 1094, // RenameUserStmt (2x) - 58487: 1095, // RepeatableOpt (2x) - 58493: 1096, // RestartStmt (2x) - 58495: 1097, // ResumeImportStmt (2x) - 57514: 1098, // revoke (2x) - 58496: 1099, // RevokeRoleStmt (2x) - 58497: 1100, // RevokeStmt (2x) - 58500: 1101, // RoleOrPrivElemList (2x) - 58501: 1102, // RoleSpec (2x) - 58522: 1103, // SelectStmtOpt (2x) - 58525: 1104, // SelectStmtSQLCache (2x) - 58529: 1105, // SetDefaultRoleOpt (2x) - 58530: 1106, // SetDefaultRoleStmt (2x) - 58540: 1107, // SetRoleStmt (2x) - 58543: 1108, // ShowImportStmt (2x) - 58548: 1109, // ShowProfileType (2x) - 58551: 1110, // ShowStmt (2x) - 58552: 1111, // ShowTableAliasOpt (2x) - 58554: 1112, // ShutdownStmt (2x) - 58555: 1113, // SignedLiteral (2x) - 58559: 1114, // SplitOption (2x) - 58560: 1115, // SplitRegionStmt (2x) - 58564: 1116, // Statement (2x) - 58567: 1117, // StatsOptionsOpt (2x) - 58568: 1118, // StatsPersistentVal (2x) - 58569: 1119, // StatsType (2x) - 58570: 1120, // StopImportStmt (2x) - 58577: 1121, // SubPartDefinition (2x) - 58580: 1122, // SubPartitionMethod (2x) - 58585: 1123, // Symbol (2x) - 58591: 1124, // TableElementList (2x) - 58594: 1125, // TableLock (2x) - 58598: 1126, // TableNameListOpt (2x) - 58605: 1127, // TableOrTables (2x) - 58614: 1128, // TablesTerminalSym (2x) - 58612: 1129, // TableToTable (2x) - 58616: 1130, // TextStringList (2x) - 58621: 1131, // TraceStmt (2x) - 58626: 1132, // TruncateTableStmt (2x) - 58629: 1133, // UnlockTablesStmt (2x) - 58635: 1134, // UserToUser (2x) - 58632: 1135, // UseStmt (2x) - 58647: 1136, // Varchar (2x) - 58650: 1137, // VariableAssignmentList (2x) - 58659: 1138, // WhenClause (2x) - 58664: 1139, // WindowDefinition (2x) - 58667: 1140, // WindowFrameBound (2x) - 58674: 1141, // WindowSpec (2x) - 58679: 1142, // WithGrantOptionOpt (2x) - 58680: 1143, // WithList (2x) - 58684: 1144, // Writeable (2x) - 58104: 1145, // AdminShowSlow (1x) - 58113: 1146, // AlterOrderList (1x) - 58116: 1147, // AlterSequenceOptionList (1x) - 58118: 1148, // AlterTablePartitionOpt (1x) - 58120: 1149, // AlterTableSpecList (1x) - 58121: 1150, // AlterTableSpecListOpt (1x) - 58125: 1151, // AnalyzeOptionList (1x) - 58128: 1152, // AnyOrAll (1x) - 58130: 1153, // AsOfClauseOpt (1x) - 58131: 1154, // AsOpt (1x) - 58136: 1155, // AuthOption (1x) - 58137: 1156, // AuthPlugin (1x) - 58148: 1157, // BetweenOrNotOp (1x) - 58152: 1158, // BitValueType (1x) - 58153: 1159, // BlobType (1x) - 58156: 1160, // BooleanType (1x) - 57370: 1161, // both (1x) - 58166: 1162, // CharsetNameOrDefault (1x) - 58167: 1163, // CharsetOpt (1x) - 58169: 1164, // ClearPasswordExpireOptions (1x) - 58173: 1165, // ColumnFormat (1x) - 58175: 1166, // ColumnList (1x) - 58182: 1167, // ColumnNameOrUserVariableList (1x) - 58179: 1168, // ColumnNameOrUserVarListOpt (1x) - 58180: 1169, // ColumnNameOrUserVarListOptWithBrackets (1x) - 58188: 1170, // ColumnSetValueList (1x) - 58192: 1171, // CompareOp (1x) - 58196: 1172, // ConnectionOptionList (1x) - 58199: 1173, // ConstraintElem (1x) - 58207: 1174, // CreateSequenceOptionListOpt (1x) - 58211: 1175, // CreateTableSelectOpt (1x) - 58214: 1176, // CreateViewSelectOpt (1x) - 58221: 1177, // DatabaseOptionListOpt (1x) - 58223: 1178, // DateAndTimeType (1x) - 58218: 1179, // DBNameList (1x) - 58229: 1180, // DefaultValueExpr (1x) - 57409: 1181, // dual (1x) - 58250: 1182, // ElseOpt (1x) - 58255: 1183, // EnforcedOrNotOrNotNullOpt (1x) - 58261: 1184, // ExplainFormatType (1x) - 58269: 1185, // ExpressionOpt (1x) - 58271: 1186, // FetchFirstOpt (1x) - 58273: 1187, // FieldAsName (1x) - 58274: 1188, // FieldAsNameOpt (1x) - 58276: 1189, // FieldItemList (1x) - 58278: 1190, // FieldList (1x) - 58284: 1191, // FirstOrNext (1x) - 58285: 1192, // FixedPointType (1x) - 58287: 1193, // FlashbackToNewName (1x) - 58289: 1194, // FloatingPointType (1x) - 58290: 1195, // FlushOption (1x) - 58293: 1196, // FromDual (1x) - 58295: 1197, // FulltextSearchModifierOpt (1x) - 58296: 1198, // FuncDatetimePrec (1x) - 58309: 1199, // GetFormatSelector (1x) - 58316: 1200, // HandleRangeList (1x) - 58318: 1201, // HavingClause (1x) - 58321: 1202, // IdentListWithParenOpt (1x) - 58325: 1203, // IfNotRunning (1x) - 58326: 1204, // IfRunning (1x) - 58327: 1205, // IgnoreLines (1x) - 58329: 1206, // ImportTruncate (1x) - 58335: 1207, // IndexHintScope (1x) - 58338: 1208, // IndexKeyTypeOpt (1x) - 58347: 1209, // IndexPartSpecificationListOpt (1x) - 58350: 1210, // IndexTypeOpt (1x) - 58330: 1211, // InOrNotOp (1x) - 58353: 1212, // InstanceOption (1x) - 58355: 1213, // IntegerType (1x) - 58358: 1214, // IsolationLevel (1x) - 58357: 1215, // IsOrNotOp (1x) - 57460: 1216, // leading (1x) - 58366: 1217, // LikeEscapeOpt (1x) - 58367: 1218, // LikeOrNotOp (1x) - 58368: 1219, // LikeTableWithOrWithoutParen (1x) - 58373: 1220, // LinesTerminated (1x) - 58376: 1221, // LoadDataSetList (1x) - 58377: 1222, // LoadDataSetSpecOpt (1x) - 58381: 1223, // LocationLabelList (1x) - 58384: 1224, // LockType (1x) - 58385: 1225, // LogTypeOpt (1x) - 58386: 1226, // Match (1x) - 58387: 1227, // MatchOpt (1x) - 58388: 1228, // MaxIndexNumOpt (1x) - 58389: 1229, // MaxMinutesOpt (1x) - 58392: 1230, // NChar (1x) - 58404: 1231, // NumericType (1x) - 58394: 1232, // NVarchar (1x) - 58409: 1233, // OnDeleteUpdateOpt (1x) - 58410: 1234, // OnDuplicateKeyUpdate (1x) - 58412: 1235, // OptBinMod (1x) - 58414: 1236, // OptCharset (1x) - 58417: 1237, // OptErrors (1x) - 58418: 1238, // OptExistingWindowName (1x) - 58420: 1239, // OptFromFirstLast (1x) - 58422: 1240, // OptGConcatSeparator (1x) - 58428: 1241, // OptPartitionClause (1x) - 58429: 1242, // OptTable (1x) - 58432: 1243, // OptWindowFrameClause (1x) - 58433: 1244, // OptWindowOrderByClause (1x) - 58438: 1245, // Order (1x) - 58437: 1246, // OrReplace (1x) - 57444: 1247, // outfile (1x) - 58444: 1248, // PartDefValuesOpt (1x) - 58448: 1249, // PartitionKeyAlgorithmOpt (1x) - 58449: 1250, // PartitionMethod (1x) - 58452: 1251, // PartitionNumOpt (1x) - 58459: 1252, // PerDB (1x) - 58460: 1253, // PerTable (1x) - 57498: 1254, // precisionType (1x) - 58468: 1255, // PrepareSQL (1x) - 58476: 1256, // ProcedureCall (1x) - 57505: 1257, // recursive (1x) - 58482: 1258, // RegexpOrNotOp (1x) - 58486: 1259, // ReorganizePartitionRuleOpt (1x) - 58491: 1260, // RequireList (1x) - 58502: 1261, // RoleSpecList (1x) - 58509: 1262, // RowOrRows (1x) - 58515: 1263, // SelectStmtFieldList (1x) - 58523: 1264, // SelectStmtOpts (1x) - 58524: 1265, // SelectStmtOptsList (1x) - 58528: 1266, // SequenceOptionList (1x) - 58532: 1267, // SetOpr (1x) - 58539: 1268, // SetRoleOpt (1x) - 58544: 1269, // ShowIndexKwd (1x) - 58545: 1270, // ShowLikeOrWhereOpt (1x) - 58546: 1271, // ShowPlacementTarget (1x) - 58547: 1272, // ShowProfileArgsOpt (1x) - 58549: 1273, // ShowProfileTypes (1x) - 58550: 1274, // ShowProfileTypesOpt (1x) - 58553: 1275, // ShowTargetFilterable (1x) - 57525: 1276, // spatial (1x) - 58561: 1277, // SplitSyntaxOption (1x) - 57530: 1278, // ssl (1x) - 58562: 1279, // Start (1x) - 58563: 1280, // Starting (1x) - 57531: 1281, // starting (1x) - 58565: 1282, // StatementList (1x) - 58566: 1283, // StatementScope (1x) - 58571: 1284, // StorageMedia (1x) - 57536: 1285, // stored (1x) - 58572: 1286, // StringList (1x) - 58575: 1287, // StringNameOrBRIEOptionKeyword (1x) - 58576: 1288, // StringType (1x) - 58578: 1289, // SubPartDefinitionList (1x) - 58579: 1290, // SubPartDefinitionListOpt (1x) - 58581: 1291, // SubPartitionNumOpt (1x) - 58582: 1292, // SubPartitionOpt (1x) - 58592: 1293, // TableElementListOpt (1x) - 58595: 1294, // TableLockList (1x) - 58608: 1295, // TableRefsClause (1x) - 58609: 1296, // TableSampleMethodOpt (1x) - 58610: 1297, // TableSampleOpt (1x) - 58611: 1298, // TableSampleUnitOpt (1x) - 58613: 1299, // TableToTableList (1x) - 58617: 1300, // TextType (1x) - 57543: 1301, // trailing (1x) - 58625: 1302, // TrimDirection (1x) - 58627: 1303, // Type (1x) - 58636: 1304, // UserToUserList (1x) - 58638: 1305, // UserVariableList (1x) - 58641: 1306, // UsingRoles (1x) - 58643: 1307, // Values (1x) - 58645: 1308, // ValuesOpt (1x) - 58652: 1309, // ViewAlgorithm (1x) - 58653: 1310, // ViewCheckOption (1x) - 58654: 1311, // ViewDefiner (1x) - 58655: 1312, // ViewFieldList (1x) - 58656: 1313, // ViewName (1x) - 58657: 1314, // ViewSQLSecurity (1x) - 57563: 1315, // virtual (1x) - 58658: 1316, // VirtualOrStored (1x) - 58660: 1317, // WhenClauseList (1x) - 58663: 1318, // WindowClauseOptional (1x) - 58665: 1319, // WindowDefinitionList (1x) - 58666: 1320, // WindowFrameBetween (1x) - 58668: 1321, // WindowFrameExtent (1x) - 58670: 1322, // WindowFrameUnits (1x) - 58673: 1323, // WindowNameOrSpec (1x) - 58675: 1324, // WindowSpecDetails (1x) - 58681: 1325, // WithReadLockOpt (1x) - 58682: 1326, // WithValidation (1x) - 58683: 1327, // WithValidationOpt (1x) - 58685: 1328, // Year (1x) - 58103: 1329, // $default (0x) - 58064: 1330, // andnot (0x) - 58134: 1331, // AssignmentListOpt (0x) - 58172: 1332, // ColumnDefList (0x) - 58189: 1333, // CommaOpt (0x) - 58087: 1334, // createTableSelect (0x) - 58078: 1335, // empty (0x) - 57345: 1336, // error (0x) - 58102: 1337, // higherThanComma (0x) - 58096: 1338, // higherThanParenthese (0x) - 58085: 1339, // insertValues (0x) - 57352: 1340, // invalid (0x) - 58088: 1341, // lowerThanCharsetKwd (0x) - 58101: 1342, // lowerThanComma (0x) - 58086: 1343, // lowerThanCreateTableSelect (0x) - 58098: 1344, // lowerThanEq (0x) - 58093: 1345, // lowerThanFunction (0x) - 58084: 1346, // lowerThanInsertValues (0x) - 58089: 1347, // lowerThanKey (0x) - 58090: 1348, // lowerThanLocal (0x) - 58100: 1349, // lowerThanNot (0x) - 58097: 1350, // lowerThanOn (0x) - 58095: 1351, // lowerThanParenthese (0x) - 58091: 1352, // lowerThanRemove (0x) - 58079: 1353, // lowerThanSelectOpt (0x) - 58083: 1354, // lowerThanSelectStmt (0x) - 58082: 1355, // lowerThanSetKeyword (0x) - 58081: 1356, // lowerThanStringLitToken (0x) - 58080: 1357, // lowerThanValueKeyword (0x) - 58092: 1358, // lowerThenOrder (0x) - 58099: 1359, // neg (0x) - 57356: 1360, // odbcDateType (0x) - 57358: 1361, // odbcTimestampType (0x) - 57357: 1362, // odbcTimeType (0x) - 58094: 1363, // tableRefPriority (0x) + 58035: 585, // builtinApproxCountDistinct (687x) + 58036: 586, // builtinApproxPercentile (687x) + 58030: 587, // builtinBitAnd (687x) + 58031: 588, // builtinBitOr (687x) + 58032: 589, // builtinBitXor (687x) + 58033: 590, // builtinCast (687x) + 58034: 591, // builtinCount (687x) + 58037: 592, // builtinCurDate (687x) + 58038: 593, // builtinCurTime (687x) + 58039: 594, // builtinDateAdd (687x) + 58040: 595, // builtinDateSub (687x) + 58041: 596, // builtinExtract (687x) + 58042: 597, // builtinGroupConcat (687x) + 58043: 598, // builtinMax (687x) + 58044: 599, // builtinMin (687x) + 58046: 600, // builtinPosition (687x) + 58050: 601, // builtinStddevPop (687x) + 58051: 602, // builtinStddevSamp (687x) + 58047: 603, // builtinSubstring (687x) + 58048: 604, // builtinSum (687x) + 58049: 605, // builtinSysDate (687x) + 58052: 606, // builtinTranslate (687x) + 58053: 607, // builtinTrim (687x) + 58054: 608, // builtinUser (687x) + 58055: 609, // builtinVarPop (687x) + 58056: 610, // builtinVarSamp (687x) + 57374: 611, // caseKwd (687x) + 57385: 612, // cumeDist (687x) + 57386: 613, // currentDate (687x) + 57390: 614, // currentRole (687x) + 57387: 615, // currentTime (687x) + 57401: 616, // denseRank (687x) + 57418: 617, // firstValue (687x) + 57457: 618, // lag (687x) + 57458: 619, // lastValue (687x) + 57459: 620, // lead (687x) + 57483: 621, // nthValue (687x) + 57484: 622, // ntile (687x) + 57497: 623, // percentRank (687x) + 57502: 624, // rank (687x) + 57510: 625, // repeat (687x) + 57519: 626, // rowNumber (687x) + 57554: 627, // utcDate (687x) + 57556: 628, // utcTime (687x) + 57555: 629, // utcTimestamp (687x) + 57546: 630, // unique (684x) + 57381: 631, // constraint (682x) + 57506: 632, // references (679x) + 57425: 633, // generated (675x) + 57521: 634, // selectKwd (667x) + 57376: 635, // character (646x) + 57473: 636, // match (637x) + 57437: 637, // index (634x) + 57542: 638, // to (556x) + 57360: 639, // all (543x) + 46: 640, // '.' (536x) + 57362: 641, // analyze (518x) + 57550: 642, // update (507x) + 58067: 643, // jss (504x) + 58068: 644, // juss (504x) + 57474: 645, // maxValue (500x) + 57464: 646, // lines (493x) + 57371: 647, // by (490x) + 58063: 648, // assignmentEq (488x) + 57512: 649, // require (485x) + 57361: 650, // alter (484x) + 58320: 651, // Identifier (483x) + 58395: 652, // NotKeywordToken (483x) + 58616: 653, // TiDBKeyword (483x) + 58626: 654, // UnReservedKeyword (483x) + 64: 655, // '@' (480x) + 57526: 656, // sql (477x) + 57408: 657, // drop (474x) + 57373: 658, // cascade (473x) + 57503: 659, // read (473x) + 57513: 660, // restrict (473x) + 57347: 661, // asof (471x) + 57383: 662, // create (469x) + 57422: 663, // foreign (469x) + 57424: 664, // fulltext (469x) + 57560: 665, // varcharacter (467x) + 57559: 666, // varcharType (467x) + 57375: 667, // change (466x) + 57397: 668, // decimalType (466x) + 57407: 669, // doubleType (466x) + 57419: 670, // floatType (466x) + 57440: 671, // integerType (466x) + 57447: 672, // intType (466x) + 57504: 673, // realType (466x) + 57509: 674, // rename (466x) + 57566: 675, // write (466x) + 57561: 676, // varbinaryType (465x) + 57359: 677, // add (464x) + 57367: 678, // bigIntType (464x) + 57369: 679, // blobType (464x) + 57448: 680, // int1Type (464x) + 57449: 681, // int2Type (464x) + 57450: 682, // int3Type (464x) + 57451: 683, // int4Type (464x) + 57452: 684, // int8Type (464x) + 57558: 685, // long (464x) + 57470: 686, // longblobType (464x) + 57471: 687, // longtextType (464x) + 57475: 688, // mediumblobType (464x) + 57476: 689, // mediumIntType (464x) + 57477: 690, // mediumtextType (464x) + 57486: 691, // numericType (464x) + 57489: 692, // optimize (464x) + 57524: 693, // smallIntType (464x) + 57539: 694, // tinyblobType (464x) + 57540: 695, // tinyIntType (464x) + 57541: 696, // tinytextType (464x) + 58581: 697, // SubSelect (209x) + 58635: 698, // UserVariable (171x) + 58556: 699, // SimpleIdent (170x) + 58372: 700, // Literal (168x) + 58571: 701, // StringLiteral (168x) + 58393: 702, // NextValueForSequence (167x) + 58297: 703, // FunctionCallGeneric (166x) + 58298: 704, // FunctionCallKeyword (166x) + 58299: 705, // FunctionCallNonKeyword (166x) + 58300: 706, // FunctionNameConflict (166x) + 58301: 707, // FunctionNameDateArith (166x) + 58302: 708, // FunctionNameDateArithMultiForms (166x) + 58303: 709, // FunctionNameDatetimePrecision (166x) + 58304: 710, // FunctionNameOptionalBraces (166x) + 58305: 711, // FunctionNameSequence (166x) + 58555: 712, // SimpleExpr (166x) + 58582: 713, // SumExpr (166x) + 58584: 714, // SystemVariable (166x) + 58646: 715, // Variable (166x) + 58669: 716, // WindowFuncCall (166x) + 58149: 717, // BitExpr (153x) + 58465: 718, // PredicateExpr (130x) + 58152: 719, // BoolPri (127x) + 58264: 720, // Expression (127x) + 58391: 721, // NUM (97x) + 58684: 722, // logAnd (96x) + 58685: 723, // logOr (96x) + 58254: 724, // EqOpt (75x) + 58594: 725, // TableName (75x) + 58572: 726, // StringName (56x) + 57549: 727, // unsigned (47x) + 57495: 728, // over (45x) + 57571: 729, // zerofill (45x) + 57400: 730, // deleteKwd (41x) + 58363: 731, // LengthNum (41x) + 58174: 732, // ColumnName (40x) + 57404: 733, // distinct (36x) + 57405: 734, // distinctRow (36x) + 58674: 735, // WindowingClause (35x) + 57399: 736, // delayed (33x) + 57430: 737, // highPriority (33x) + 57472: 738, // lowPriority (33x) + 58511: 739, // SelectStmt (30x) + 58512: 740, // SelectStmtBasic (30x) + 58514: 741, // SelectStmtFromDualTable (30x) + 58515: 742, // SelectStmtFromTable (30x) + 58531: 743, // SetOprClause (30x) + 58532: 744, // SetOprClauseList (29x) + 58535: 745, // SetOprStmtWithLimitOrderBy (29x) + 58536: 746, // SetOprStmtWoutLimitOrderBy (29x) + 57353: 747, // hintComment (27x) + 58275: 748, // FieldLen (26x) + 58352: 749, // Int64Num (26x) + 58524: 750, // SelectStmtWithClause (26x) + 58534: 751, // SetOprStmt (26x) + 58675: 752, // WithClause (26x) + 58432: 753, // OptWindowingClause (24x) + 58437: 754, // OrderBy (23x) + 58518: 755, // SelectStmtLimit (23x) + 57527: 756, // sqlBigResult (23x) + 57528: 757, // sqlCalcFoundRows (23x) + 57529: 758, // sqlSmallResult (23x) + 58162: 759, // CharsetKw (20x) + 58637: 760, // Username (20x) + 58629: 761, // UpdateStmtNoWith (18x) + 58230: 762, // DeleteWithoutUsingStmt (17x) + 58265: 763, // ExpressionList (17x) + 58460: 764, // PlacementPolicyOption (17x) + 58321: 765, // IfExists (16x) + 58349: 766, // InsertIntoStmt (16x) + 58486: 767, // ReplaceIntoStmt (16x) + 57537: 768, // terminated (16x) + 58628: 769, // UpdateStmt (16x) + 58232: 770, // DistinctKwd (15x) + 58322: 771, // IfNotExists (15x) + 58417: 772, // OptFieldLen (15x) + 58233: 773, // DistinctOpt (14x) + 57411: 774, // enclosed (14x) + 58448: 775, // PartitionNameList (14x) + 58659: 776, // WhereClause (14x) + 58660: 777, // WhereClauseOptional (14x) + 58225: 778, // DefaultKwdOpt (13x) + 58229: 779, // DeleteWithUsingStmt (13x) + 57412: 780, // escaped (13x) + 57491: 781, // optionally (13x) + 58595: 782, // TableNameList (13x) + 58228: 783, // DeleteFromStmt (12x) + 58263: 784, // ExprOrDefault (12x) + 58357: 785, // JoinTable (12x) + 58411: 786, // OptBinary (12x) + 58502: 787, // RolenameComposed (12x) + 58591: 788, // TableFactor (12x) + 58604: 789, // TableRef (12x) + 58124: 790, // AnalyzeOptionListOpt (11x) + 58292: 791, // FromOrIn (11x) + 58618: 792, // TimestampUnit (11x) + 58163: 793, // CharsetName (10x) + 58175: 794, // ColumnNameList (10x) + 57466: 795, // load (10x) + 58396: 796, // NotSym (10x) + 58438: 797, // OrderByOptional (10x) + 58440: 798, // PartDefOption (10x) + 58554: 799, // SignedNum (10x) + 58155: 800, // BuggyDefaultFalseDistinctOpt (9x) + 58215: 801, // DBName (9x) + 58224: 802, // DefaultFalseDistinctOpt (9x) + 58358: 803, // JoinType (9x) + 57482: 804, // noWriteToBinLog (9x) + 58401: 805, // NumLiteral (9x) + 58501: 806, // Rolename (9x) + 58496: 807, // RoleNameString (9x) + 58120: 808, // AlterTableStmt (8x) + 58214: 809, // CrossOpt (8x) + 58255: 810, // EqOrAssignmentEq (8x) + 58266: 811, // ExpressionListOpt (8x) + 58343: 812, // IndexPartSpecification (8x) + 58359: 813, // KeyOrIndex (8x) + 58519: 814, // SelectStmtLimitOpt (8x) + 58617: 815, // TimeUnit (8x) + 58649: 816, // VariableName (8x) + 58106: 817, // AllOrPartitionNameList (7x) + 58198: 818, // ConstraintKeywordOpt (7x) + 58281: 819, // FieldsOrColumns (7x) + 58290: 820, // ForceOpt (7x) + 58344: 821, // IndexPartSpecificationList (7x) + 58394: 822, // NoWriteToBinLogAliasOpt (7x) + 58469: 823, // Priority (7x) + 58506: 824, // RowFormat (7x) + 58509: 825, // RowValue (7x) + 58529: 826, // SetExpr (7x) + 58540: 827, // ShowDatabaseNameOpt (7x) + 58601: 828, // TableOption (7x) + 57562: 829, // varying (7x) + 58145: 830, // BeginTransactionStmt (6x) + 57380: 831, // column (6x) + 58169: 832, // ColumnDef (6x) + 58188: 833, // CommitStmt (6x) + 58217: 834, // DatabaseOption (6x) + 58220: 835, // DatabaseSym (6x) + 58257: 836, // EscapedTableRef (6x) + 58262: 837, // ExplainableStmt (6x) + 58279: 838, // FieldTerminator (6x) + 57426: 839, // grant (6x) + 58326: 840, // IgnoreOptional (6x) + 58335: 841, // IndexInvisible (6x) + 58340: 842, // IndexNameList (6x) + 58346: 843, // IndexType (6x) + 58376: 844, // LoadDataStmt (6x) + 58449: 845, // PartitionNameListOpt (6x) + 57508: 846, // release (6x) + 58503: 847, // RolenameList (6x) + 58505: 848, // RollbackStmt (6x) + 58539: 849, // SetStmt (6x) + 57523: 850, // show (6x) + 58599: 851, // TableOptimizerHints (6x) + 58638: 852, // UsernameList (6x) + 58676: 853, // WithClustered (6x) + 58104: 854, // AlgorithmClause (5x) + 58156: 855, // ByItem (5x) + 58168: 856, // CollationName (5x) + 58172: 857, // ColumnKeywordOpt (5x) + 58231: 858, // DirectPlacementOption (5x) + 58277: 859, // FieldOpt (5x) + 58278: 860, // FieldOpts (5x) + 58318: 861, // IdentList (5x) + 58338: 862, // IndexName (5x) + 58341: 863, // IndexOption (5x) + 58342: 864, // IndexOptionList (5x) + 57438: 865, // infile (5x) + 58368: 866, // LimitOption (5x) + 58380: 867, // LockClause (5x) + 58413: 868, // OptCharsetWithOptBinary (5x) + 58424: 869, // OptNullTreatment (5x) + 58463: 870, // PolicyName (5x) + 58470: 871, // PriorityOpt (5x) + 58510: 872, // SelectLockOpt (5x) + 58517: 873, // SelectStmtIntoOption (5x) + 58605: 874, // TableRefs (5x) + 58631: 875, // UserSpec (5x) + 58130: 876, // Assignment (4x) + 58136: 877, // AuthString (4x) + 58147: 878, // BindableStmt (4x) + 58137: 879, // BRIEBooleanOptionName (4x) + 58138: 880, // BRIEIntegerOptionName (4x) + 58139: 881, // BRIEKeywordOptionName (4x) + 58140: 882, // BRIEOption (4x) + 58141: 883, // BRIEOptions (4x) + 58143: 884, // BRIEStringOptionName (4x) + 58157: 885, // ByList (4x) + 58161: 886, // Char (4x) + 58192: 887, // ConfigItemName (4x) + 58196: 888, // Constraint (4x) + 58286: 889, // FloatOpt (4x) + 58347: 890, // IndexTypeName (4x) + 57490: 891, // option (4x) + 58429: 892, // OptWild (4x) + 57494: 893, // outer (4x) + 58464: 894, // Precision (4x) + 58478: 895, // ReferDef (4x) + 58492: 896, // RestrictOrCascadeOpt (4x) + 58508: 897, // RowStmt (4x) + 58525: 898, // SequenceOption (4x) + 57532: 899, // statsExtended (4x) + 58586: 900, // TableAsName (4x) + 58587: 901, // TableAsNameOpt (4x) + 58598: 902, // TableNameOptWild (4x) + 58600: 903, // TableOptimizerHintsOpt (4x) + 58602: 904, // TableOptionList (4x) + 58620: 905, // TraceableStmt (4x) + 58621: 906, // TransactionChar (4x) + 58632: 907, // UserSpecList (4x) + 58670: 908, // WindowName (4x) + 58127: 909, // AsOfClause (3x) + 58131: 910, // AssignmentList (3x) + 58133: 911, // AttributesOpt (3x) + 58153: 912, // Boolean (3x) + 58181: 913, // ColumnOption (3x) + 58184: 914, // ColumnPosition (3x) + 58189: 915, // CommonTableExpr (3x) + 58210: 916, // CreateTableStmt (3x) + 58218: 917, // DatabaseOptionList (3x) + 58226: 918, // DefaultTrueDistinctOpt (3x) + 58251: 919, // EnforcedOrNot (3x) + 57414: 920, // explain (3x) + 58268: 921, // ExtendedPriv (3x) + 58306: 922, // GeneratedAlways (3x) + 58308: 923, // GlobalScope (3x) + 58312: 924, // GroupByClause (3x) + 58330: 925, // IndexHint (3x) + 58334: 926, // IndexHintType (3x) + 58339: 927, // IndexNameAndTypeOpt (3x) + 57455: 928, // keys (3x) + 58370: 929, // Lines (3x) + 58388: 930, // MaxValueOrExpression (3x) + 58425: 931, // OptOrder (3x) + 58428: 932, // OptTemporary (3x) + 58441: 933, // PartDefOptionList (3x) + 58443: 934, // PartitionDefinition (3x) + 58452: 935, // PasswordExpire (3x) + 58454: 936, // PasswordOrLockOption (3x) + 58462: 937, // PluginNameList (3x) + 58468: 938, // PrimaryOpt (3x) + 58471: 939, // PrivElem (3x) + 58473: 940, // PrivType (3x) + 57500: 941, // procedure (3x) + 58487: 942, // RequireClause (3x) + 58488: 943, // RequireClauseOpt (3x) + 58490: 944, // RequireListElement (3x) + 58504: 945, // RolenameWithoutIdent (3x) + 58497: 946, // RoleOrPrivElem (3x) + 58516: 947, // SelectStmtGroup (3x) + 58533: 948, // SetOprOpt (3x) + 58585: 949, // TableAliasRefList (3x) + 58588: 950, // TableElement (3x) + 58597: 951, // TableNameListOpt2 (3x) + 58613: 952, // TextString (3x) + 58622: 953, // TransactionChars (3x) + 57544: 954, // trigger (3x) + 57548: 955, // unlock (3x) + 57551: 956, // usage (3x) + 58642: 957, // ValuesList (3x) + 58644: 958, // ValuesStmtList (3x) + 58640: 959, // ValueSym (3x) + 58647: 960, // VariableAssignment (3x) + 58667: 961, // WindowFrameStart (3x) + 58103: 962, // AdminStmt (2x) + 58105: 963, // AllColumnsOrPredicateColumnsOpt (2x) + 58107: 964, // AlterDatabaseStmt (2x) + 58108: 965, // AlterImportStmt (2x) + 58109: 966, // AlterInstanceStmt (2x) + 58110: 967, // AlterOrderItem (2x) + 58112: 968, // AlterPolicyStmt (2x) + 58113: 969, // AlterSequenceOption (2x) + 58115: 970, // AlterSequenceStmt (2x) + 58117: 971, // AlterTableSpec (2x) + 58121: 972, // AlterUserStmt (2x) + 58122: 973, // AnalyzeOption (2x) + 58125: 974, // AnalyzeTableStmt (2x) + 58148: 975, // BinlogStmt (2x) + 58142: 976, // BRIEStmt (2x) + 58144: 977, // BRIETables (2x) + 57372: 978, // call (2x) + 58158: 979, // CallStmt (2x) + 58159: 980, // CastType (2x) + 58160: 981, // ChangeStmt (2x) + 58166: 982, // CheckConstraintKeyword (2x) + 58176: 983, // ColumnNameListOpt (2x) + 58179: 984, // ColumnNameOrUserVariable (2x) + 58182: 985, // ColumnOptionList (2x) + 58183: 986, // ColumnOptionListOpt (2x) + 58185: 987, // ColumnSetValue (2x) + 58191: 988, // CompletionTypeWithinTransaction (2x) + 58193: 989, // ConnectionOption (2x) + 58195: 990, // ConnectionOptions (2x) + 58199: 991, // CreateBindingStmt (2x) + 58200: 992, // CreateDatabaseStmt (2x) + 58201: 993, // CreateImportStmt (2x) + 58202: 994, // CreateIndexStmt (2x) + 58203: 995, // CreatePolicyStmt (2x) + 58204: 996, // CreateRoleStmt (2x) + 58206: 997, // CreateSequenceStmt (2x) + 58207: 998, // CreateStatisticsStmt (2x) + 58208: 999, // CreateTableOptionListOpt (2x) + 58211: 1000, // CreateUserStmt (2x) + 58213: 1001, // CreateViewStmt (2x) + 57392: 1002, // databases (2x) + 58222: 1003, // DeallocateStmt (2x) + 58223: 1004, // DeallocateSym (2x) + 57403: 1005, // describe (2x) + 58234: 1006, // DoStmt (2x) + 58235: 1007, // DropBindingStmt (2x) + 58236: 1008, // DropDatabaseStmt (2x) + 58237: 1009, // DropImportStmt (2x) + 58238: 1010, // DropIndexStmt (2x) + 58239: 1011, // DropPolicyStmt (2x) + 58240: 1012, // DropRoleStmt (2x) + 58241: 1013, // DropSequenceStmt (2x) + 58242: 1014, // DropStatisticsStmt (2x) + 58243: 1015, // DropStatsStmt (2x) + 58244: 1016, // DropTableStmt (2x) + 58245: 1017, // DropUserStmt (2x) + 58246: 1018, // DropViewStmt (2x) + 58247: 1019, // DuplicateOpt (2x) + 58249: 1020, // EmptyStmt (2x) + 58250: 1021, // EncryptionOpt (2x) + 58252: 1022, // EnforcedOrNotOpt (2x) + 58256: 1023, // ErrorHandling (2x) + 58258: 1024, // ExecuteStmt (2x) + 58260: 1025, // ExplainStmt (2x) + 58261: 1026, // ExplainSym (2x) + 58270: 1027, // Field (2x) + 58273: 1028, // FieldItem (2x) + 58280: 1029, // Fields (2x) + 58284: 1030, // FlashbackTableStmt (2x) + 58289: 1031, // FlushStmt (2x) + 58295: 1032, // FuncDatetimePrecList (2x) + 58296: 1033, // FuncDatetimePrecListOpt (2x) + 58309: 1034, // GrantProxyStmt (2x) + 58310: 1035, // GrantRoleStmt (2x) + 58311: 1036, // GrantStmt (2x) + 58313: 1037, // HandleRange (2x) + 58315: 1038, // HashString (2x) + 58317: 1039, // HelpStmt (2x) + 58329: 1040, // IndexAdviseStmt (2x) + 58331: 1041, // IndexHintList (2x) + 58332: 1042, // IndexHintListOpt (2x) + 58337: 1043, // IndexLockAndAlgorithmOpt (2x) + 58350: 1044, // InsertValues (2x) + 58354: 1045, // IntoOpt (2x) + 58360: 1046, // KeyOrIndexOpt (2x) + 57456: 1047, // kill (2x) + 58361: 1048, // KillOrKillTiDB (2x) + 58362: 1049, // KillStmt (2x) + 58367: 1050, // LimitClause (2x) + 57465: 1051, // linear (2x) + 58369: 1052, // LinearOpt (2x) + 58373: 1053, // LoadDataSetItem (2x) + 58377: 1054, // LoadStatsStmt (2x) + 58378: 1055, // LocalOpt (2x) + 58379: 1056, // LocationLabelList (2x) + 58381: 1057, // LockTablesStmt (2x) + 58389: 1058, // MaxValueOrExpressionList (2x) + 58397: 1059, // NowSym (2x) + 58398: 1060, // NowSymFunc (2x) + 58399: 1061, // NowSymOptionFraction (2x) + 58400: 1062, // NumList (2x) + 58403: 1063, // ObjectType (2x) + 57487: 1064, // of (2x) + 58404: 1065, // OfTablesOpt (2x) + 58405: 1066, // OnCommitOpt (2x) + 58406: 1067, // OnDelete (2x) + 58409: 1068, // OnUpdate (2x) + 58414: 1069, // OptCollate (2x) + 58419: 1070, // OptFull (2x) + 58421: 1071, // OptInteger (2x) + 58434: 1072, // OptionalBraces (2x) + 58433: 1073, // OptionLevel (2x) + 58423: 1074, // OptLeadLagInfo (2x) + 58422: 1075, // OptLLDefault (2x) + 58439: 1076, // OuterOpt (2x) + 58444: 1077, // PartitionDefinitionList (2x) + 58445: 1078, // PartitionDefinitionListOpt (2x) + 58451: 1079, // PartitionOpt (2x) + 58453: 1080, // PasswordOpt (2x) + 58455: 1081, // PasswordOrLockOptionList (2x) + 58456: 1082, // PasswordOrLockOptions (2x) + 58459: 1083, // PlacementOptionList (2x) + 58461: 1084, // PlanReplayerStmt (2x) + 58467: 1085, // PreparedStmt (2x) + 58472: 1086, // PrivLevel (2x) + 58475: 1087, // PurgeImportStmt (2x) + 58476: 1088, // QuickOptional (2x) + 58477: 1089, // RecoverTableStmt (2x) + 58479: 1090, // ReferOpt (2x) + 58481: 1091, // RegexpSym (2x) + 58482: 1092, // RenameTableStmt (2x) + 58483: 1093, // RenameUserStmt (2x) + 58485: 1094, // RepeatableOpt (2x) + 58491: 1095, // RestartStmt (2x) + 58493: 1096, // ResumeImportStmt (2x) + 57514: 1097, // revoke (2x) + 58494: 1098, // RevokeRoleStmt (2x) + 58495: 1099, // RevokeStmt (2x) + 58498: 1100, // RoleOrPrivElemList (2x) + 58499: 1101, // RoleSpec (2x) + 58520: 1102, // SelectStmtOpt (2x) + 58523: 1103, // SelectStmtSQLCache (2x) + 58527: 1104, // SetDefaultRoleOpt (2x) + 58528: 1105, // SetDefaultRoleStmt (2x) + 58538: 1106, // SetRoleStmt (2x) + 58541: 1107, // ShowImportStmt (2x) + 58546: 1108, // ShowProfileType (2x) + 58549: 1109, // ShowStmt (2x) + 58550: 1110, // ShowTableAliasOpt (2x) + 58552: 1111, // ShutdownStmt (2x) + 58553: 1112, // SignedLiteral (2x) + 58557: 1113, // SplitOption (2x) + 58558: 1114, // SplitRegionStmt (2x) + 58562: 1115, // Statement (2x) + 58565: 1116, // StatsOptionsOpt (2x) + 58566: 1117, // StatsPersistentVal (2x) + 58567: 1118, // StatsType (2x) + 58568: 1119, // StopImportStmt (2x) + 58575: 1120, // SubPartDefinition (2x) + 58578: 1121, // SubPartitionMethod (2x) + 58583: 1122, // Symbol (2x) + 58589: 1123, // TableElementList (2x) + 58592: 1124, // TableLock (2x) + 58596: 1125, // TableNameListOpt (2x) + 58603: 1126, // TableOrTables (2x) + 58612: 1127, // TablesTerminalSym (2x) + 58610: 1128, // TableToTable (2x) + 58614: 1129, // TextStringList (2x) + 58619: 1130, // TraceStmt (2x) + 58624: 1131, // TruncateTableStmt (2x) + 58627: 1132, // UnlockTablesStmt (2x) + 58633: 1133, // UserToUser (2x) + 58630: 1134, // UseStmt (2x) + 58645: 1135, // Varchar (2x) + 58648: 1136, // VariableAssignmentList (2x) + 58657: 1137, // WhenClause (2x) + 58662: 1138, // WindowDefinition (2x) + 58665: 1139, // WindowFrameBound (2x) + 58672: 1140, // WindowSpec (2x) + 58677: 1141, // WithGrantOptionOpt (2x) + 58678: 1142, // WithList (2x) + 58682: 1143, // Writeable (2x) + 58102: 1144, // AdminShowSlow (1x) + 58111: 1145, // AlterOrderList (1x) + 58114: 1146, // AlterSequenceOptionList (1x) + 58116: 1147, // AlterTablePartitionOpt (1x) + 58118: 1148, // AlterTableSpecList (1x) + 58119: 1149, // AlterTableSpecListOpt (1x) + 58123: 1150, // AnalyzeOptionList (1x) + 58126: 1151, // AnyOrAll (1x) + 58128: 1152, // AsOfClauseOpt (1x) + 58129: 1153, // AsOpt (1x) + 58134: 1154, // AuthOption (1x) + 58135: 1155, // AuthPlugin (1x) + 58146: 1156, // BetweenOrNotOp (1x) + 58150: 1157, // BitValueType (1x) + 58151: 1158, // BlobType (1x) + 58154: 1159, // BooleanType (1x) + 57370: 1160, // both (1x) + 58164: 1161, // CharsetNameOrDefault (1x) + 58165: 1162, // CharsetOpt (1x) + 58167: 1163, // ClearPasswordExpireOptions (1x) + 58171: 1164, // ColumnFormat (1x) + 58173: 1165, // ColumnList (1x) + 58180: 1166, // ColumnNameOrUserVariableList (1x) + 58177: 1167, // ColumnNameOrUserVarListOpt (1x) + 58178: 1168, // ColumnNameOrUserVarListOptWithBrackets (1x) + 58186: 1169, // ColumnSetValueList (1x) + 58190: 1170, // CompareOp (1x) + 58194: 1171, // ConnectionOptionList (1x) + 58197: 1172, // ConstraintElem (1x) + 58205: 1173, // CreateSequenceOptionListOpt (1x) + 58209: 1174, // CreateTableSelectOpt (1x) + 58212: 1175, // CreateViewSelectOpt (1x) + 58219: 1176, // DatabaseOptionListOpt (1x) + 58221: 1177, // DateAndTimeType (1x) + 58216: 1178, // DBNameList (1x) + 58227: 1179, // DefaultValueExpr (1x) + 57409: 1180, // dual (1x) + 58248: 1181, // ElseOpt (1x) + 58253: 1182, // EnforcedOrNotOrNotNullOpt (1x) + 58259: 1183, // ExplainFormatType (1x) + 58267: 1184, // ExpressionOpt (1x) + 58269: 1185, // FetchFirstOpt (1x) + 58271: 1186, // FieldAsName (1x) + 58272: 1187, // FieldAsNameOpt (1x) + 58274: 1188, // FieldItemList (1x) + 58276: 1189, // FieldList (1x) + 58282: 1190, // FirstOrNext (1x) + 58283: 1191, // FixedPointType (1x) + 58285: 1192, // FlashbackToNewName (1x) + 58287: 1193, // FloatingPointType (1x) + 58288: 1194, // FlushOption (1x) + 58291: 1195, // FromDual (1x) + 58293: 1196, // FulltextSearchModifierOpt (1x) + 58294: 1197, // FuncDatetimePrec (1x) + 58307: 1198, // GetFormatSelector (1x) + 58314: 1199, // HandleRangeList (1x) + 58316: 1200, // HavingClause (1x) + 58319: 1201, // IdentListWithParenOpt (1x) + 58323: 1202, // IfNotRunning (1x) + 58324: 1203, // IfRunning (1x) + 58325: 1204, // IgnoreLines (1x) + 58327: 1205, // ImportTruncate (1x) + 58333: 1206, // IndexHintScope (1x) + 58336: 1207, // IndexKeyTypeOpt (1x) + 58345: 1208, // IndexPartSpecificationListOpt (1x) + 58348: 1209, // IndexTypeOpt (1x) + 58328: 1210, // InOrNotOp (1x) + 58351: 1211, // InstanceOption (1x) + 58353: 1212, // IntegerType (1x) + 58356: 1213, // IsolationLevel (1x) + 58355: 1214, // IsOrNotOp (1x) + 57460: 1215, // leading (1x) + 58364: 1216, // LikeEscapeOpt (1x) + 58365: 1217, // LikeOrNotOp (1x) + 58366: 1218, // LikeTableWithOrWithoutParen (1x) + 58371: 1219, // LinesTerminated (1x) + 58374: 1220, // LoadDataSetList (1x) + 58375: 1221, // LoadDataSetSpecOpt (1x) + 58382: 1222, // LockType (1x) + 58383: 1223, // LogTypeOpt (1x) + 58384: 1224, // Match (1x) + 58385: 1225, // MatchOpt (1x) + 58386: 1226, // MaxIndexNumOpt (1x) + 58387: 1227, // MaxMinutesOpt (1x) + 58390: 1228, // NChar (1x) + 58402: 1229, // NumericType (1x) + 58392: 1230, // NVarchar (1x) + 58407: 1231, // OnDeleteUpdateOpt (1x) + 58408: 1232, // OnDuplicateKeyUpdate (1x) + 58410: 1233, // OptBinMod (1x) + 58412: 1234, // OptCharset (1x) + 58415: 1235, // OptErrors (1x) + 58416: 1236, // OptExistingWindowName (1x) + 58418: 1237, // OptFromFirstLast (1x) + 58420: 1238, // OptGConcatSeparator (1x) + 58426: 1239, // OptPartitionClause (1x) + 58427: 1240, // OptTable (1x) + 58430: 1241, // OptWindowFrameClause (1x) + 58431: 1242, // OptWindowOrderByClause (1x) + 58436: 1243, // Order (1x) + 58435: 1244, // OrReplace (1x) + 57444: 1245, // outfile (1x) + 58442: 1246, // PartDefValuesOpt (1x) + 58446: 1247, // PartitionKeyAlgorithmOpt (1x) + 58447: 1248, // PartitionMethod (1x) + 58450: 1249, // PartitionNumOpt (1x) + 58457: 1250, // PerDB (1x) + 58458: 1251, // PerTable (1x) + 57498: 1252, // precisionType (1x) + 58466: 1253, // PrepareSQL (1x) + 58474: 1254, // ProcedureCall (1x) + 57505: 1255, // recursive (1x) + 58480: 1256, // RegexpOrNotOp (1x) + 58484: 1257, // ReorganizePartitionRuleOpt (1x) + 58489: 1258, // RequireList (1x) + 58500: 1259, // RoleSpecList (1x) + 58507: 1260, // RowOrRows (1x) + 58513: 1261, // SelectStmtFieldList (1x) + 58521: 1262, // SelectStmtOpts (1x) + 58522: 1263, // SelectStmtOptsList (1x) + 58526: 1264, // SequenceOptionList (1x) + 58530: 1265, // SetOpr (1x) + 58537: 1266, // SetRoleOpt (1x) + 58542: 1267, // ShowIndexKwd (1x) + 58543: 1268, // ShowLikeOrWhereOpt (1x) + 58544: 1269, // ShowPlacementTarget (1x) + 58545: 1270, // ShowProfileArgsOpt (1x) + 58547: 1271, // ShowProfileTypes (1x) + 58548: 1272, // ShowProfileTypesOpt (1x) + 58551: 1273, // ShowTargetFilterable (1x) + 57525: 1274, // spatial (1x) + 58559: 1275, // SplitSyntaxOption (1x) + 57530: 1276, // ssl (1x) + 58560: 1277, // Start (1x) + 58561: 1278, // Starting (1x) + 57531: 1279, // starting (1x) + 58563: 1280, // StatementList (1x) + 58564: 1281, // StatementScope (1x) + 58569: 1282, // StorageMedia (1x) + 57536: 1283, // stored (1x) + 58570: 1284, // StringList (1x) + 58573: 1285, // StringNameOrBRIEOptionKeyword (1x) + 58574: 1286, // StringType (1x) + 58576: 1287, // SubPartDefinitionList (1x) + 58577: 1288, // SubPartDefinitionListOpt (1x) + 58579: 1289, // SubPartitionNumOpt (1x) + 58580: 1290, // SubPartitionOpt (1x) + 58590: 1291, // TableElementListOpt (1x) + 58593: 1292, // TableLockList (1x) + 58606: 1293, // TableRefsClause (1x) + 58607: 1294, // TableSampleMethodOpt (1x) + 58608: 1295, // TableSampleOpt (1x) + 58609: 1296, // TableSampleUnitOpt (1x) + 58611: 1297, // TableToTableList (1x) + 58615: 1298, // TextType (1x) + 57543: 1299, // trailing (1x) + 58623: 1300, // TrimDirection (1x) + 58625: 1301, // Type (1x) + 58634: 1302, // UserToUserList (1x) + 58636: 1303, // UserVariableList (1x) + 58639: 1304, // UsingRoles (1x) + 58641: 1305, // Values (1x) + 58643: 1306, // ValuesOpt (1x) + 58650: 1307, // ViewAlgorithm (1x) + 58651: 1308, // ViewCheckOption (1x) + 58652: 1309, // ViewDefiner (1x) + 58653: 1310, // ViewFieldList (1x) + 58654: 1311, // ViewName (1x) + 58655: 1312, // ViewSQLSecurity (1x) + 57563: 1313, // virtual (1x) + 58656: 1314, // VirtualOrStored (1x) + 58658: 1315, // WhenClauseList (1x) + 58661: 1316, // WindowClauseOptional (1x) + 58663: 1317, // WindowDefinitionList (1x) + 58664: 1318, // WindowFrameBetween (1x) + 58666: 1319, // WindowFrameExtent (1x) + 58668: 1320, // WindowFrameUnits (1x) + 58671: 1321, // WindowNameOrSpec (1x) + 58673: 1322, // WindowSpecDetails (1x) + 58679: 1323, // WithReadLockOpt (1x) + 58680: 1324, // WithValidation (1x) + 58681: 1325, // WithValidationOpt (1x) + 58683: 1326, // Year (1x) + 58101: 1327, // $default (0x) + 58062: 1328, // andnot (0x) + 58132: 1329, // AssignmentListOpt (0x) + 58170: 1330, // ColumnDefList (0x) + 58187: 1331, // CommaOpt (0x) + 58085: 1332, // createTableSelect (0x) + 58076: 1333, // empty (0x) + 57345: 1334, // error (0x) + 58100: 1335, // higherThanComma (0x) + 58094: 1336, // higherThanParenthese (0x) + 58083: 1337, // insertValues (0x) + 57352: 1338, // invalid (0x) + 58086: 1339, // lowerThanCharsetKwd (0x) + 58099: 1340, // lowerThanComma (0x) + 58084: 1341, // lowerThanCreateTableSelect (0x) + 58096: 1342, // lowerThanEq (0x) + 58091: 1343, // lowerThanFunction (0x) + 58082: 1344, // lowerThanInsertValues (0x) + 58087: 1345, // lowerThanKey (0x) + 58088: 1346, // lowerThanLocal (0x) + 58098: 1347, // lowerThanNot (0x) + 58095: 1348, // lowerThanOn (0x) + 58093: 1349, // lowerThanParenthese (0x) + 58089: 1350, // lowerThanRemove (0x) + 58077: 1351, // lowerThanSelectOpt (0x) + 58081: 1352, // lowerThanSelectStmt (0x) + 58080: 1353, // lowerThanSetKeyword (0x) + 58079: 1354, // lowerThanStringLitToken (0x) + 58078: 1355, // lowerThanValueKeyword (0x) + 58090: 1356, // lowerThenOrder (0x) + 58097: 1357, // neg (0x) + 57356: 1358, // odbcDateType (0x) + 57358: 1359, // odbcTimestampType (0x) + 57357: 1360, // odbcTimeType (0x) + 58092: 1361, // tableRefPriority (0x) } yySymNames = []string{ @@ -2207,9 +2203,9 @@ var ( "placement", "keyBlockSize", "tablespace", + "encryption", "engine", "data", - "encryption", "insertMethod", "maxRows", "minRows", @@ -2364,6 +2360,7 @@ var ( "fixed", "isolation", "jsonType", + "location", "max_idxnum", "memory", "off", @@ -2389,7 +2386,6 @@ var ( "flush", "full", "identSQLErrors", - "location", "mb", "mode", "never", @@ -2455,6 +2451,7 @@ var ( "rebuild", "redundant", "reload", + "replica", "restore", "routine", "s3", @@ -2469,6 +2466,7 @@ var ( "statsOptions", "stop", "swaps", + "tiFlash", "tokudbDefault", "tokudbFast", "tokudbLzma", @@ -2554,7 +2552,6 @@ var ( "recent", "region", "replayer", - "replica", "reset", "restores", "security", @@ -2573,7 +2570,6 @@ var ( "temptable", "textType", "than", - "tiFlash", "tls", "top", "traditional", @@ -2681,6 +2677,7 @@ var ( "order", "values", "force", + "set", "and", "charType", "replace", @@ -2689,7 +2686,6 @@ var ( "andand", "pipesAsOr", "xor", - "set", "group", "straightJoin", "window", @@ -2720,9 +2716,9 @@ var ( "secondMicrosecond", "yearMonth", "when", - "binaryType", "in", "elseKwd", + "binaryType", "then", "'<'", "'>'", @@ -2773,7 +2769,6 @@ var ( "underscoreCS", "'!'", "'~'", - "builtinAddDate", "builtinApproxCountDistinct", "builtinApproxPercentile", "builtinBitAnd", @@ -2792,7 +2787,6 @@ var ( "builtinPosition", "builtinStddevPop", "builtinStddevSamp", - "builtinSubDate", "builtinSubstring", "builtinSum", "builtinSysDate", @@ -2911,9 +2905,9 @@ var ( "PredicateExpr", "BoolPri", "Expression", + "NUM", "logAnd", "logOr", - "NUM", "EqOpt", "TableName", "StringName", @@ -2921,8 +2915,8 @@ var ( "over", "zerofill", "deleteKwd", - "ColumnName", "LengthNum", + "ColumnName", "distinct", "distinctRow", "WindowingClause", @@ -3246,6 +3240,7 @@ var ( "LoadDataSetItem", "LoadStatsStmt", "LocalOpt", + "LocationLabelList", "LockTablesStmt", "MaxValueOrExpressionList", "NowSym", @@ -3411,7 +3406,6 @@ var ( "LinesTerminated", "LoadDataSetList", "LoadDataSetSpecOpt", - "LocationLabelList", "LockType", "LogTypeOpt", "Match", @@ -3556,1051 +3550,679 @@ var ( yyReductions = []struct{ xsym, components int }{ {0, 1}, - {1279, 1}, - {810, 6}, - {810, 8}, - {810, 10}, - {1084, 1}, - {1084, 2}, - {1084, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {860, 3}, - {766, 4}, - {766, 4}, - {766, 4}, - {766, 4}, - {913, 3}, - {913, 3}, - {1117, 3}, - {1117, 3}, + {1277, 1}, + {808, 6}, + {808, 8}, + {808, 10}, + {1083, 1}, + {1083, 2}, + {1083, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {858, 3}, + {764, 4}, + {764, 4}, + {764, 4}, + {764, 4}, + {911, 3}, + {911, 3}, + {1116, 3}, + {1116, 3}, + {1147, 1}, + {1147, 2}, + {1147, 4}, + {1147, 3}, + {1147, 3}, + {1056, 0}, + {1056, 3}, + {971, 1}, + {971, 5}, + {971, 5}, + {971, 5}, + {971, 5}, + {971, 6}, + {971, 2}, + {971, 5}, + {971, 6}, + {971, 8}, + {971, 1}, + {971, 1}, + {971, 3}, + {971, 4}, + {971, 5}, + {971, 3}, + {971, 4}, + {971, 4}, + {971, 7}, + {971, 3}, + {971, 4}, + {971, 4}, + {971, 4}, + {971, 4}, + {971, 2}, + {971, 2}, + {971, 4}, + {971, 4}, + {971, 5}, + {971, 3}, + {971, 2}, + {971, 2}, + {971, 5}, + {971, 6}, + {971, 6}, + {971, 8}, + {971, 5}, + {971, 5}, + {971, 3}, + {971, 3}, + {971, 3}, + {971, 5}, + {971, 1}, + {971, 1}, + {971, 1}, + {971, 1}, + {971, 2}, + {971, 2}, + {971, 1}, + {971, 1}, + {971, 4}, + {971, 3}, + {971, 4}, + {971, 1}, + {971, 1}, + {1257, 0}, + {1257, 5}, + {817, 1}, + {817, 1}, + {1325, 0}, + {1325, 1}, + {1324, 2}, + {1324, 2}, + {853, 1}, + {853, 1}, + {854, 3}, + {854, 3}, + {854, 3}, + {854, 3}, + {854, 3}, + {867, 3}, + {867, 3}, + {1143, 2}, + {1143, 2}, + {813, 1}, + {813, 1}, + {1046, 0}, + {1046, 1}, + {857, 0}, + {857, 1}, + {914, 0}, + {914, 1}, + {914, 2}, + {1149, 0}, + {1149, 1}, {1148, 1}, - {1148, 2}, - {1148, 4}, {1148, 3}, - {1148, 3}, - {1223, 0}, - {1223, 3}, - {973, 1}, - {973, 5}, - {973, 5}, - {973, 5}, - {973, 5}, - {973, 6}, - {973, 2}, - {973, 5}, - {973, 6}, - {973, 8}, - {973, 1}, - {973, 1}, - {973, 3}, - {973, 4}, - {973, 5}, - {973, 3}, - {973, 4}, - {973, 4}, - {973, 7}, - {973, 3}, - {973, 4}, - {973, 4}, - {973, 4}, - {973, 4}, - {973, 2}, - {973, 2}, - {973, 4}, - {973, 4}, - {973, 5}, - {973, 3}, + {775, 1}, + {775, 3}, + {818, 0}, + {818, 1}, + {818, 2}, + {1122, 1}, + {1092, 3}, + {1297, 1}, + {1297, 3}, + {1128, 3}, + {1093, 3}, + {1302, 1}, + {1302, 3}, + {1133, 3}, + {1089, 5}, + {1089, 3}, + {1089, 4}, + {1030, 4}, + {1192, 0}, + {1192, 2}, + {1114, 6}, + {1114, 8}, + {1113, 6}, + {1113, 2}, + {1275, 0}, + {1275, 2}, + {1275, 1}, + {1275, 3}, + {974, 5}, + {974, 6}, + {974, 7}, + {974, 7}, + {974, 8}, + {974, 9}, + {974, 8}, + {974, 7}, + {974, 6}, + {974, 8}, + {963, 0}, + {963, 2}, + {963, 2}, + {790, 0}, + {790, 2}, + {1150, 1}, + {1150, 3}, {973, 2}, {973, 2}, - {973, 5}, - {973, 6}, - {973, 6}, - {973, 8}, - {973, 5}, - {973, 5}, - {973, 3}, {973, 3}, {973, 3}, - {973, 5}, - {973, 1}, - {973, 1}, - {973, 1}, - {973, 1}, {973, 2}, {973, 2}, - {973, 1}, - {973, 1}, - {973, 4}, - {973, 3}, - {973, 4}, - {973, 1}, - {973, 1}, - {1259, 0}, - {1259, 5}, - {819, 1}, - {819, 1}, - {1327, 0}, - {1327, 1}, - {1326, 2}, - {1326, 2}, - {855, 1}, - {855, 1}, - {856, 3}, - {856, 3}, - {856, 3}, - {856, 3}, - {856, 3}, - {869, 3}, - {869, 3}, - {1144, 2}, - {1144, 2}, - {815, 1}, - {815, 1}, - {1048, 0}, - {1048, 1}, - {859, 0}, - {859, 1}, - {916, 0}, - {916, 1}, - {916, 2}, - {1150, 0}, - {1150, 1}, - {1149, 1}, - {1149, 3}, - {777, 1}, - {777, 3}, - {820, 0}, - {820, 1}, - {820, 2}, - {1123, 1}, - {1093, 3}, - {1299, 1}, - {1299, 3}, - {1129, 3}, - {1094, 3}, - {1304, 1}, - {1304, 3}, - {1134, 3}, - {1090, 5}, - {1090, 3}, - {1090, 4}, - {1032, 4}, - {1193, 0}, - {1193, 2}, - {1115, 6}, - {1115, 8}, - {1114, 6}, - {1114, 2}, - {1277, 0}, - {1277, 2}, - {1277, 1}, - {1277, 3}, - {976, 5}, - {976, 6}, - {976, 7}, - {976, 7}, - {976, 8}, - {976, 9}, - {976, 8}, - {976, 7}, - {976, 6}, - {976, 8}, - {965, 0}, - {965, 2}, - {965, 2}, - {792, 0}, - {792, 2}, - {1151, 1}, - {1151, 3}, - {975, 2}, - {975, 2}, - {975, 3}, - {975, 3}, - {975, 2}, + {876, 3}, + {910, 1}, + {910, 3}, + {1329, 0}, + {1329, 1}, + {830, 1}, + {830, 2}, + {830, 2}, + {830, 2}, + {830, 4}, + {830, 5}, + {830, 6}, + {830, 4}, + {830, 5}, {975, 2}, - {878, 3}, - {912, 1}, - {912, 3}, - {1331, 0}, - {1331, 1}, - {832, 1}, - {832, 2}, - {832, 2}, - {832, 2}, - {832, 4}, - {832, 5}, - {832, 6}, - {832, 4}, - {832, 5}, - {977, 2}, - {1332, 1}, - {1332, 3}, - {834, 3}, - {834, 3}, - {733, 1}, - {733, 3}, - {733, 5}, + {1330, 1}, + {1330, 3}, + {832, 3}, + {832, 3}, + {732, 1}, + {732, 3}, + {732, 5}, + {794, 1}, + {794, 3}, + {983, 0}, + {983, 1}, + {1201, 0}, + {1201, 3}, + {861, 1}, + {861, 3}, + {1167, 0}, + {1167, 1}, + {1166, 1}, + {1166, 3}, + {984, 1}, + {984, 1}, + {1168, 0}, + {1168, 3}, + {833, 1}, + {833, 2}, + {938, 0}, + {938, 1}, + {796, 1}, {796, 1}, - {796, 3}, - {985, 0}, + {919, 1}, + {919, 2}, + {1022, 0}, + {1022, 1}, + {1182, 2}, + {1182, 1}, + {913, 2}, + {913, 1}, + {913, 1}, + {913, 2}, + {913, 3}, + {913, 1}, + {913, 2}, + {913, 2}, + {913, 3}, + {913, 3}, + {913, 2}, + {913, 6}, + {913, 6}, + {913, 1}, + {913, 2}, + {913, 2}, + {913, 2}, + {913, 2}, + {1282, 1}, + {1282, 1}, + {1282, 1}, + {1164, 1}, + {1164, 1}, + {1164, 1}, + {922, 0}, + {922, 2}, + {1314, 0}, + {1314, 1}, + {1314, 1}, {985, 1}, - {1202, 0}, - {1202, 3}, - {863, 1}, - {863, 3}, - {1168, 0}, - {1168, 1}, - {1167, 1}, - {1167, 3}, - {986, 1}, + {985, 2}, + {986, 0}, {986, 1}, - {1169, 0}, - {1169, 3}, - {835, 1}, - {835, 2}, - {940, 0}, - {940, 1}, - {798, 1}, - {798, 1}, - {921, 1}, - {921, 2}, - {1024, 0}, - {1024, 1}, - {1183, 2}, - {1183, 1}, - {915, 2}, - {915, 1}, - {915, 1}, - {915, 2}, - {915, 3}, - {915, 1}, - {915, 2}, - {915, 2}, - {915, 3}, - {915, 3}, - {915, 2}, - {915, 6}, - {915, 6}, - {915, 1}, - {915, 2}, - {915, 2}, - {915, 2}, - {915, 2}, - {1284, 1}, - {1284, 1}, - {1284, 1}, - {1165, 1}, - {1165, 1}, - {1165, 1}, - {924, 0}, - {924, 2}, - {1316, 0}, - {1316, 1}, - {1316, 1}, - {987, 1}, - {987, 2}, - {988, 0}, - {988, 1}, - {1173, 7}, - {1173, 7}, - {1173, 7}, - {1173, 7}, - {1173, 8}, - {1173, 5}, - {1226, 2}, - {1226, 2}, - {1226, 2}, - {1227, 0}, - {1227, 1}, - {897, 5}, + {1172, 7}, + {1172, 7}, + {1172, 7}, + {1172, 7}, + {1172, 8}, + {1172, 5}, + {1224, 2}, + {1224, 2}, + {1224, 2}, + {1225, 0}, + {1225, 1}, + {895, 5}, + {1067, 3}, {1068, 3}, - {1069, 3}, - {1233, 0}, - {1233, 1}, - {1233, 1}, - {1233, 2}, - {1233, 2}, - {1091, 1}, - {1091, 1}, - {1091, 2}, - {1091, 2}, - {1091, 2}, - {1180, 1}, - {1180, 1}, - {1180, 1}, - {1062, 1}, - {1062, 3}, - {1062, 4}, - {704, 4}, - {704, 4}, - {1061, 1}, - {1061, 1}, - {1061, 1}, + {1231, 0}, + {1231, 1}, + {1231, 1}, + {1231, 2}, + {1231, 2}, + {1090, 1}, + {1090, 1}, + {1090, 2}, + {1090, 2}, + {1090, 2}, + {1179, 1}, + {1179, 1}, + {1179, 1}, {1061, 1}, + {1061, 3}, + {1061, 4}, + {702, 4}, + {702, 4}, {1060, 1}, {1060, 1}, {1060, 1}, - {1113, 1}, - {1113, 2}, - {1113, 2}, - {807, 1}, - {807, 1}, - {807, 1}, - {1119, 1}, - {1119, 1}, - {1119, 1}, - {1000, 12}, - {1016, 3}, - {996, 13}, - {1209, 0}, - {1209, 3}, - {823, 1}, - {823, 3}, - {814, 3}, - {814, 4}, - {1045, 0}, - {1045, 1}, - {1045, 1}, - {1045, 2}, - {1045, 2}, + {1060, 1}, + {1059, 1}, + {1059, 1}, + {1059, 1}, + {1112, 1}, + {1112, 2}, + {1112, 2}, + {805, 1}, + {805, 1}, + {805, 1}, + {1118, 1}, + {1118, 1}, + {1118, 1}, + {998, 12}, + {1014, 3}, + {994, 13}, {1208, 0}, - {1208, 1}, - {1208, 1}, - {1208, 1}, - {966, 4}, - {966, 3}, - {994, 5}, - {803, 1}, - {872, 1}, - {836, 4}, - {836, 4}, - {836, 4}, - {836, 2}, - {836, 1}, - {1177, 0}, - {1177, 1}, - {919, 1}, - {919, 2}, - {918, 12}, - {918, 7}, - {1067, 0}, - {1067, 4}, - {1067, 4}, - {780, 0}, - {780, 1}, - {1080, 0}, - {1080, 6}, - {1122, 6}, - {1122, 5}, - {1249, 0}, - {1249, 3}, - {1250, 1}, - {1250, 4}, - {1250, 5}, - {1250, 4}, - {1250, 5}, - {1250, 4}, - {1250, 3}, - {1250, 1}, - {1054, 0}, - {1054, 1}, - {1292, 0}, - {1292, 4}, - {1291, 0}, - {1291, 2}, - {1251, 0}, - {1251, 2}, + {1208, 3}, + {821, 1}, + {821, 3}, + {812, 3}, + {812, 4}, + {1043, 0}, + {1043, 1}, + {1043, 1}, + {1043, 2}, + {1043, 2}, + {1207, 0}, + {1207, 1}, + {1207, 1}, + {1207, 1}, + {964, 4}, + {964, 3}, + {992, 5}, + {801, 1}, + {870, 1}, + {834, 4}, + {834, 4}, + {834, 4}, + {834, 2}, + {834, 1}, + {834, 5}, + {1176, 0}, + {1176, 1}, + {917, 1}, + {917, 2}, + {916, 12}, + {916, 7}, + {1066, 0}, + {1066, 4}, + {1066, 4}, + {778, 0}, + {778, 1}, {1079, 0}, - {1079, 3}, - {1078, 1}, - {1078, 3}, - {936, 5}, - {1290, 0}, - {1290, 3}, - {1289, 1}, - {1289, 3}, - {1121, 3}, - {935, 0}, - {935, 2}, - {800, 3}, - {800, 3}, - {800, 4}, - {800, 3}, - {800, 4}, - {800, 4}, - {800, 3}, - {800, 3}, - {800, 3}, - {800, 3}, - {800, 1}, - {1248, 0}, - {1248, 4}, - {1248, 6}, + {1079, 6}, + {1121, 6}, + {1121, 5}, + {1247, 0}, + {1247, 3}, {1248, 1}, + {1248, 4}, {1248, 5}, + {1248, 4}, + {1248, 5}, + {1248, 4}, + {1248, 3}, {1248, 1}, - {1248, 1}, - {1021, 0}, - {1021, 1}, - {1021, 1}, - {1154, 0}, - {1154, 1}, - {1175, 0}, + {1052, 0}, + {1052, 1}, + {1290, 0}, + {1290, 4}, + {1289, 0}, + {1289, 2}, + {1249, 0}, + {1249, 2}, + {1078, 0}, + {1078, 3}, + {1077, 1}, + {1077, 3}, + {934, 5}, + {1288, 0}, + {1288, 3}, + {1287, 1}, + {1287, 3}, + {1120, 3}, + {933, 0}, + {933, 2}, + {798, 3}, + {798, 3}, + {798, 4}, + {798, 3}, + {798, 4}, + {798, 4}, + {798, 3}, + {798, 3}, + {798, 3}, + {798, 3}, + {798, 1}, + {1246, 0}, + {1246, 4}, + {1246, 6}, + {1246, 1}, + {1246, 5}, + {1246, 1}, + {1246, 1}, + {1019, 0}, + {1019, 1}, + {1019, 1}, + {1153, 0}, + {1153, 1}, + {1174, 0}, + {1174, 1}, + {1174, 1}, + {1174, 1}, + {1174, 1}, {1175, 1}, {1175, 1}, {1175, 1}, {1175, 1}, - {1176, 1}, - {1176, 1}, - {1176, 1}, - {1176, 1}, - {1219, 2}, - {1219, 4}, - {1003, 11}, - {1246, 0}, - {1246, 2}, + {1218, 2}, + {1218, 4}, + {1001, 11}, + {1244, 0}, + {1244, 2}, + {1307, 0}, + {1307, 3}, + {1307, 3}, + {1307, 3}, {1309, 0}, {1309, 3}, - {1309, 3}, - {1309, 3}, - {1311, 0}, - {1311, 3}, - {1314, 0}, - {1314, 3}, - {1314, 3}, - {1313, 1}, {1312, 0}, {1312, 3}, - {1166, 1}, - {1166, 3}, + {1312, 3}, + {1311, 1}, {1310, 0}, - {1310, 4}, - {1310, 4}, - {1008, 2}, - {764, 13}, - {764, 9}, - {781, 10}, - {785, 1}, - {785, 1}, - {785, 2}, - {785, 2}, - {837, 1}, - {1010, 4}, - {1012, 7}, + {1310, 3}, + {1165, 1}, + {1165, 3}, + {1308, 0}, + {1308, 4}, + {1308, 4}, + {1006, 2}, + {762, 13}, + {762, 9}, + {779, 10}, + {783, 1}, + {783, 1}, + {783, 2}, + {783, 2}, + {835, 1}, + {1008, 4}, + {1010, 7}, + {1016, 6}, + {932, 0}, + {932, 1}, + {932, 2}, + {1018, 4}, {1018, 6}, - {934, 0}, - {934, 1}, - {934, 2}, - {1020, 4}, - {1020, 6}, - {1019, 3}, - {1019, 5}, - {1014, 3}, - {1014, 5}, {1017, 3}, {1017, 5}, - {1017, 4}, - {898, 0}, - {898, 1}, - {898, 1}, - {1127, 1}, - {1127, 1}, - {726, 0}, - {726, 1}, - {1022, 0}, - {1131, 2}, - {1131, 5}, - {1131, 3}, - {1131, 6}, - {1028, 1}, - {1028, 1}, - {1028, 1}, - {1027, 2}, - {1027, 3}, - {1027, 2}, - {1027, 4}, - {1027, 7}, - {1027, 5}, - {1027, 7}, - {1027, 5}, - {1027, 3}, - {1184, 1}, - {1184, 1}, - {1184, 1}, - {1184, 1}, - {1184, 1}, - {1184, 1}, - {978, 5}, - {978, 5}, - {979, 2}, - {979, 2}, - {979, 2}, - {1179, 1}, - {1179, 3}, - {885, 0}, - {885, 2}, - {882, 1}, - {882, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {881, 1}, - {886, 1}, - {886, 1}, - {886, 1}, - {886, 1}, - {883, 1}, - {883, 1}, - {883, 2}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 5}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 6}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 3}, - {884, 3}, - {734, 1}, - {751, 1}, - {725, 1}, - {914, 1}, - {914, 1}, - {914, 1}, - {1074, 1}, - {1074, 1}, - {1074, 1}, - {1088, 3}, - {995, 8}, - {1120, 4}, - {1097, 4}, - {967, 6}, - {1011, 4}, - {1108, 5}, - {1204, 0}, - {1204, 2}, - {1203, 0}, - {1203, 3}, - {1237, 0}, - {1237, 1}, - {1025, 0}, - {1025, 1}, - {1025, 2}, - {1025, 2}, + {1012, 3}, + {1012, 5}, + {1015, 3}, + {1015, 5}, + {1015, 4}, + {896, 0}, + {896, 1}, + {896, 1}, + {1126, 1}, + {1126, 1}, + {724, 0}, + {724, 1}, + {1020, 0}, + {1130, 2}, + {1130, 5}, + {1130, 3}, + {1130, 6}, + {1026, 1}, + {1026, 1}, + {1026, 1}, {1025, 2}, + {1025, 3}, {1025, 2}, - {1206, 0}, - {1206, 3}, - {1206, 3}, - {722, 3}, - {722, 3}, - {722, 3}, - {722, 3}, - {722, 2}, - {722, 9}, - {722, 3}, - {722, 3}, - {722, 3}, - {722, 1}, - {932, 1}, - {932, 1}, - {1197, 0}, - {1197, 4}, - {1197, 7}, - {1197, 3}, - {1197, 3}, - {724, 1}, - {724, 1}, - {723, 1}, - {723, 1}, - {765, 1}, - {765, 3}, - {1059, 1}, - {1059, 3}, - {813, 0}, - {813, 1}, - {1035, 0}, - {1035, 1}, - {1034, 1}, - {721, 3}, - {721, 3}, - {721, 4}, - {721, 5}, + {1025, 4}, + {1025, 7}, + {1025, 5}, + {1025, 7}, + {1025, 5}, + {1025, 3}, + {1183, 1}, + {1183, 1}, + {1183, 1}, + {1183, 1}, + {1183, 1}, + {1183, 1}, + {976, 5}, + {976, 5}, + {977, 2}, + {977, 2}, + {977, 2}, + {1178, 1}, + {1178, 3}, + {883, 0}, + {883, 2}, + {880, 1}, + {880, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {879, 1}, + {884, 1}, + {884, 1}, + {884, 1}, + {884, 1}, + {881, 1}, + {881, 1}, + {881, 2}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 5}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 6}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {882, 3}, + {731, 1}, + {749, 1}, {721, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1171, 1}, - {1157, 1}, - {1157, 2}, - {1215, 1}, - {1215, 2}, - {1211, 1}, - {1211, 2}, - {1218, 1}, - {1218, 2}, - {1258, 1}, - {1258, 2}, - {1152, 1}, - {1152, 1}, - {1152, 1}, - {720, 5}, + {912, 1}, + {912, 1}, + {912, 1}, + {1073, 1}, + {1073, 1}, + {1073, 1}, + {1087, 3}, + {993, 8}, + {1119, 4}, + {1096, 4}, + {965, 6}, + {1009, 4}, + {1107, 5}, + {1203, 0}, + {1203, 2}, + {1202, 0}, + {1202, 3}, + {1235, 0}, + {1235, 1}, + {1023, 0}, + {1023, 1}, + {1023, 2}, + {1023, 2}, + {1023, 2}, + {1023, 2}, + {1205, 0}, + {1205, 3}, + {1205, 3}, + {720, 3}, + {720, 3}, + {720, 3}, + {720, 3}, + {720, 2}, + {720, 9}, + {720, 3}, {720, 3}, - {720, 5}, - {720, 4}, {720, 3}, {720, 1}, - {1092, 1}, - {1092, 1}, - {1217, 0}, + {930, 1}, + {930, 1}, + {1196, 0}, + {1196, 4}, + {1196, 7}, + {1196, 3}, + {1196, 3}, + {723, 1}, + {723, 1}, + {722, 1}, + {722, 1}, + {763, 1}, + {763, 3}, + {1058, 1}, + {1058, 3}, + {811, 0}, + {811, 1}, + {1033, 0}, + {1033, 1}, + {1032, 1}, + {719, 3}, + {719, 3}, + {719, 4}, + {719, 5}, + {719, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1170, 1}, + {1156, 1}, + {1156, 2}, + {1214, 1}, + {1214, 2}, + {1210, 1}, + {1210, 2}, + {1217, 1}, {1217, 2}, - {1029, 1}, - {1029, 3}, - {1029, 5}, - {1029, 2}, - {1188, 0}, - {1188, 1}, - {1187, 1}, - {1187, 2}, + {1256, 1}, + {1256, 2}, + {1151, 1}, + {1151, 1}, + {1151, 1}, + {718, 5}, + {718, 3}, + {718, 5}, + {718, 4}, + {718, 3}, + {718, 1}, + {1091, 1}, + {1091, 1}, + {1216, 0}, + {1216, 2}, + {1027, 1}, + {1027, 3}, + {1027, 5}, + {1027, 2}, + {1187, 0}, {1187, 1}, - {1187, 2}, - {1190, 1}, - {1190, 3}, - {926, 3}, - {1201, 0}, - {1201, 2}, - {1153, 0}, - {1153, 1}, - {911, 3}, - {767, 0}, - {767, 2}, - {773, 0}, - {773, 3}, - {842, 0}, - {842, 1}, + {1186, 1}, + {1186, 2}, + {1186, 1}, + {1186, 2}, + {1189, 1}, + {1189, 3}, + {924, 3}, + {1200, 0}, + {1200, 2}, + {1152, 0}, + {1152, 1}, + {909, 3}, + {765, 0}, + {765, 2}, + {771, 0}, + {771, 3}, + {840, 0}, + {840, 1}, + {862, 0}, + {862, 1}, {864, 0}, - {864, 1}, - {866, 0}, - {866, 2}, - {865, 3}, - {865, 1}, - {865, 3}, - {865, 2}, - {865, 1}, - {865, 1}, - {929, 1}, - {929, 3}, - {929, 3}, - {1210, 0}, - {1210, 1}, - {845, 2}, - {845, 2}, - {892, 1}, - {892, 1}, - {892, 1}, - {843, 1}, - {843, 1}, - {653, 1}, - {653, 1}, - {653, 1}, - {653, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {656, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, - {655, 1}, + {864, 2}, + {863, 3}, + {863, 1}, + {863, 3}, + {863, 2}, + {863, 1}, + {863, 1}, + {927, 1}, + {927, 3}, + {927, 3}, + {1209, 0}, + {1209, 1}, + {843, 2}, + {843, 2}, + {890, 1}, + {890, 1}, + {890, 1}, + {841, 1}, + {841, 1}, + {651, 1}, + {651, 1}, + {651, 1}, + {651, 1}, {654, 1}, {654, 1}, {654, 1}, @@ -4684,1412 +4306,1785 @@ var ( {654, 1}, {654, 1}, {654, 1}, - {981, 2}, - {1256, 1}, - {1256, 3}, - {1256, 4}, - {1256, 6}, - {768, 9}, - {1047, 0}, - {1047, 1}, - {1046, 5}, - {1046, 4}, - {1046, 4}, - {1046, 4}, - {1046, 4}, - {1046, 2}, - {1046, 1}, - {1046, 1}, - {1046, 1}, - {1046, 1}, - {1046, 2}, - {961, 1}, - {961, 1}, - {959, 1}, - {959, 3}, - {827, 3}, - {1308, 0}, - {1308, 1}, - {1307, 3}, - {1307, 1}, - {786, 1}, - {786, 1}, - {989, 3}, - {1170, 0}, - {1170, 1}, - {1170, 3}, - {1234, 0}, - {1234, 5}, - {769, 6}, - {702, 1}, - {702, 1}, - {702, 1}, - {702, 1}, - {702, 1}, - {702, 1}, - {702, 1}, - {702, 2}, - {702, 1}, - {702, 1}, - {702, 2}, - {702, 2}, - {703, 1}, - {703, 2}, - {1146, 1}, - {1146, 3}, - {969, 2}, - {756, 3}, - {887, 1}, - {887, 3}, - {857, 1}, - {857, 2}, - {1245, 1}, - {1245, 1}, - {933, 0}, - {933, 1}, - {933, 1}, - {799, 0}, - {799, 1}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 5}, - {719, 5}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 3}, - {719, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {654, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {653, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {652, 1}, + {979, 2}, + {1254, 1}, + {1254, 3}, + {1254, 4}, + {1254, 6}, + {766, 9}, + {1045, 0}, + {1045, 1}, + {1044, 5}, + {1044, 4}, + {1044, 4}, + {1044, 4}, + {1044, 4}, + {1044, 2}, + {1044, 1}, + {1044, 1}, + {1044, 1}, + {1044, 1}, + {1044, 2}, + {959, 1}, + {959, 1}, + {957, 1}, + {957, 3}, + {825, 3}, + {1306, 0}, + {1306, 1}, + {1305, 3}, + {1305, 1}, + {784, 1}, + {784, 1}, + {987, 3}, + {1169, 0}, + {1169, 1}, + {1169, 3}, + {1232, 0}, + {1232, 5}, + {767, 6}, + {700, 1}, + {700, 1}, + {700, 1}, + {700, 1}, + {700, 1}, + {700, 1}, + {700, 1}, + {700, 2}, + {700, 1}, + {700, 1}, + {700, 2}, + {700, 2}, {701, 1}, - {701, 3}, - {701, 5}, - {714, 1}, - {714, 1}, - {714, 1}, - {714, 1}, - {714, 3}, - {714, 1}, - {714, 1}, - {714, 1}, - {714, 1}, - {714, 1}, - {714, 2}, - {714, 2}, - {714, 2}, - {714, 2}, - {714, 3}, - {714, 2}, - {714, 1}, - {714, 3}, - {714, 5}, - {714, 6}, - {714, 2}, - {714, 4}, - {714, 2}, - {714, 6}, - {714, 5}, - {714, 6}, - {714, 6}, - {714, 4}, - {714, 4}, - {714, 3}, - {714, 3}, - {772, 1}, - {772, 1}, - {775, 1}, - {775, 1}, - {804, 0}, - {804, 1}, - {920, 0}, - {920, 1}, - {802, 1}, - {802, 2}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {708, 1}, - {1073, 0}, - {1073, 2}, + {701, 2}, + {1145, 1}, + {1145, 3}, + {967, 2}, + {754, 3}, + {885, 1}, + {885, 3}, + {855, 1}, + {855, 2}, + {1243, 1}, + {1243, 1}, + {931, 0}, + {931, 1}, + {931, 1}, + {797, 0}, + {797, 1}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 5}, + {717, 5}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 3}, + {717, 1}, + {699, 1}, + {699, 3}, + {699, 5}, {712, 1}, {712, 1}, {712, 1}, {712, 1}, - {711, 1}, - {711, 1}, - {711, 1}, - {711, 1}, - {711, 1}, - {711, 1}, - {706, 4}, - {706, 4}, - {706, 2}, - {706, 3}, - {706, 2}, - {706, 4}, - {706, 6}, - {706, 2}, - {706, 2}, - {706, 2}, - {706, 4}, - {706, 6}, - {706, 4}, - {707, 4}, - {707, 4}, - {707, 6}, - {707, 8}, - {707, 8}, - {707, 6}, - {707, 6}, - {707, 6}, - {707, 6}, - {707, 6}, - {707, 8}, - {707, 8}, - {707, 8}, - {707, 8}, - {707, 4}, - {707, 6}, - {707, 6}, - {707, 7}, - {707, 4}, - {707, 7}, - {707, 7}, - {707, 1}, - {707, 8}, - {1199, 1}, - {1199, 1}, - {1199, 1}, - {1199, 1}, - {709, 1}, - {709, 1}, + {712, 3}, + {712, 1}, + {712, 1}, + {712, 1}, + {712, 1}, + {712, 1}, + {712, 2}, + {712, 2}, + {712, 2}, + {712, 2}, + {712, 3}, + {712, 2}, + {712, 1}, + {712, 3}, + {712, 5}, + {712, 6}, + {712, 2}, + {712, 4}, + {712, 2}, + {712, 6}, + {712, 5}, + {712, 6}, + {712, 6}, + {712, 4}, + {712, 4}, + {712, 3}, + {712, 3}, + {770, 1}, + {770, 1}, + {773, 1}, + {773, 1}, + {802, 0}, + {802, 1}, + {918, 0}, + {918, 1}, + {800, 1}, + {800, 2}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {706, 1}, + {1072, 0}, + {1072, 2}, {710, 1}, {710, 1}, - {1302, 1}, - {1302, 1}, - {1302, 1}, - {713, 4}, - {713, 6}, - {713, 1}, - {715, 6}, - {715, 4}, - {715, 4}, - {715, 5}, - {715, 6}, - {715, 5}, - {715, 6}, - {715, 5}, - {715, 6}, - {715, 5}, - {715, 6}, - {715, 5}, - {715, 5}, - {715, 8}, - {715, 6}, - {715, 6}, - {715, 6}, - {715, 6}, - {715, 6}, - {715, 6}, - {715, 6}, - {715, 5}, - {715, 6}, - {715, 7}, - {715, 8}, - {715, 8}, - {715, 9}, - {1240, 0}, - {1240, 2}, + {710, 1}, + {710, 1}, + {709, 1}, + {709, 1}, + {709, 1}, + {709, 1}, + {709, 1}, + {709, 1}, + {704, 4}, + {704, 4}, + {704, 2}, + {704, 3}, + {704, 2}, + {704, 4}, + {704, 6}, + {704, 2}, + {704, 2}, + {704, 2}, + {704, 4}, + {704, 6}, + {704, 4}, {705, 4}, - {705, 6}, - {1198, 0}, - {1198, 2}, - {1198, 3}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {817, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {794, 1}, - {1185, 0}, - {1185, 1}, - {1317, 1}, - {1317, 2}, - {1138, 4}, - {1182, 0}, - {1182, 2}, - {982, 2}, - {982, 3}, - {982, 1}, - {982, 1}, - {982, 2}, - {982, 2}, - {982, 2}, - {982, 2}, - {982, 2}, - {982, 1}, - {982, 1}, - {982, 2}, - {982, 1}, - {825, 1}, - {825, 1}, - {825, 1}, - {873, 0}, - {873, 1}, - {727, 1}, - {727, 3}, - {784, 1}, - {784, 3}, - {904, 2}, - {904, 4}, - {951, 1}, - {951, 3}, - {894, 0}, - {894, 2}, - {1089, 0}, - {1089, 1}, - {1086, 4}, - {1255, 1}, - {1255, 1}, - {1026, 2}, - {1026, 4}, - {1305, 1}, - {1305, 3}, - {1005, 3}, - {1006, 1}, - {1006, 1}, - {850, 1}, - {850, 2}, - {990, 4}, - {990, 4}, - {990, 5}, - {990, 2}, - {990, 3}, - {990, 1}, - {990, 2}, - {1112, 1}, - {1096, 1}, - {1041, 2}, - {742, 3}, - {743, 3}, - {744, 7}, - {1297, 0}, - {1297, 7}, - {1297, 5}, + {705, 4}, + {705, 6}, + {705, 8}, + {705, 8}, + {705, 6}, + {705, 6}, + {705, 6}, + {705, 6}, + {705, 6}, + {705, 8}, + {705, 8}, + {705, 8}, + {705, 8}, + {705, 4}, + {705, 6}, + {705, 6}, + {705, 7}, + {705, 4}, + {705, 7}, + {705, 7}, + {705, 1}, + {705, 8}, + {1198, 1}, + {1198, 1}, + {1198, 1}, + {1198, 1}, + {707, 1}, + {707, 1}, + {708, 1}, + {708, 1}, + {1300, 1}, + {1300, 1}, + {1300, 1}, + {711, 4}, + {711, 6}, + {711, 1}, + {713, 6}, + {713, 4}, + {713, 4}, + {713, 5}, + {713, 6}, + {713, 5}, + {713, 6}, + {713, 5}, + {713, 6}, + {713, 5}, + {713, 6}, + {713, 5}, + {713, 5}, + {713, 8}, + {713, 6}, + {713, 6}, + {713, 6}, + {713, 6}, + {713, 6}, + {713, 6}, + {713, 6}, + {713, 5}, + {713, 6}, + {713, 7}, + {713, 8}, + {713, 8}, + {713, 9}, + {1238, 0}, + {1238, 2}, + {703, 4}, + {703, 6}, + {1197, 0}, + {1197, 2}, + {1197, 3}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {815, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {792, 1}, + {1184, 0}, + {1184, 1}, + {1315, 1}, + {1315, 2}, + {1137, 4}, + {1181, 0}, + {1181, 2}, + {980, 2}, + {980, 3}, + {980, 1}, + {980, 1}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 2}, + {980, 1}, + {980, 1}, + {980, 2}, + {980, 1}, + {823, 1}, + {823, 1}, + {823, 1}, + {871, 0}, + {871, 1}, + {725, 1}, + {725, 3}, + {782, 1}, + {782, 3}, + {902, 2}, + {902, 4}, + {949, 1}, + {949, 3}, + {892, 0}, + {892, 2}, + {1088, 0}, + {1088, 1}, + {1085, 4}, + {1253, 1}, + {1253, 1}, + {1024, 2}, + {1024, 4}, + {1303, 1}, + {1303, 3}, + {1003, 3}, + {1004, 1}, + {1004, 1}, + {848, 1}, + {848, 2}, + {988, 4}, + {988, 4}, + {988, 5}, + {988, 2}, + {988, 3}, + {988, 1}, + {988, 2}, + {1111, 1}, + {1095, 1}, + {1039, 2}, + {740, 3}, + {741, 3}, + {742, 7}, + {1295, 0}, + {1295, 7}, + {1295, 5}, + {1294, 0}, + {1294, 1}, + {1294, 1}, + {1294, 1}, {1296, 0}, {1296, 1}, {1296, 1}, - {1296, 1}, - {1298, 0}, - {1298, 1}, - {1298, 1}, - {1095, 0}, - {1095, 4}, - {741, 7}, - {741, 6}, - {741, 5}, - {741, 6}, - {741, 6}, + {1094, 0}, + {1094, 4}, + {739, 7}, + {739, 6}, + {739, 5}, + {739, 6}, + {739, 6}, + {750, 2}, + {750, 2}, {752, 2}, - {752, 2}, - {754, 2}, - {754, 3}, - {1143, 3}, - {1143, 1}, - {917, 4}, - {1196, 2}, - {1318, 0}, - {1318, 2}, - {1319, 1}, - {1319, 3}, - {1139, 3}, - {910, 1}, - {1141, 3}, - {1324, 4}, - {1238, 0}, - {1238, 1}, + {752, 3}, + {1142, 3}, + {1142, 1}, + {915, 4}, + {1195, 2}, + {1316, 0}, + {1316, 2}, + {1317, 1}, + {1317, 3}, + {1138, 3}, + {908, 1}, + {1140, 3}, + {1322, 4}, + {1236, 0}, + {1236, 1}, + {1239, 0}, + {1239, 3}, + {1242, 0}, + {1242, 3}, {1241, 0}, - {1241, 3}, - {1244, 0}, - {1244, 3}, - {1243, 0}, - {1243, 2}, - {1322, 1}, - {1322, 1}, - {1322, 1}, + {1241, 2}, + {1320, 1}, + {1320, 1}, + {1320, 1}, + {1319, 1}, + {1319, 1}, + {961, 2}, + {961, 2}, + {961, 2}, + {961, 4}, + {961, 2}, + {1318, 4}, + {1139, 1}, + {1139, 2}, + {1139, 2}, + {1139, 2}, + {1139, 4}, + {753, 0}, + {753, 1}, + {735, 2}, {1321, 1}, {1321, 1}, - {963, 2}, - {963, 2}, - {963, 2}, - {963, 4}, - {963, 2}, - {1320, 4}, - {1140, 1}, - {1140, 2}, - {1140, 2}, - {1140, 2}, - {1140, 4}, - {755, 0}, - {755, 1}, - {737, 2}, - {1323, 1}, - {1323, 1}, - {718, 4}, - {718, 4}, - {718, 4}, - {718, 4}, - {718, 4}, - {718, 5}, - {718, 7}, - {718, 7}, - {718, 6}, - {718, 6}, - {718, 9}, + {716, 4}, + {716, 4}, + {716, 4}, + {716, 4}, + {716, 4}, + {716, 5}, + {716, 7}, + {716, 7}, + {716, 6}, + {716, 6}, + {716, 9}, + {1074, 0}, + {1074, 3}, + {1074, 3}, {1075, 0}, - {1075, 3}, - {1075, 3}, + {1075, 2}, + {869, 0}, + {869, 2}, + {869, 2}, + {1237, 0}, + {1237, 2}, + {1237, 2}, + {1293, 1}, + {874, 1}, + {874, 3}, + {836, 1}, + {836, 4}, + {789, 1}, + {789, 1}, + {788, 6}, + {788, 2}, + {788, 3}, + {845, 0}, + {845, 4}, + {901, 0}, + {901, 1}, + {900, 1}, + {900, 2}, + {926, 2}, + {926, 2}, + {926, 2}, + {1206, 0}, + {1206, 2}, + {1206, 3}, + {1206, 3}, + {925, 5}, + {842, 0}, + {842, 1}, + {842, 3}, + {842, 1}, + {842, 3}, + {1041, 1}, + {1041, 2}, + {1042, 0}, + {1042, 1}, + {785, 3}, + {785, 5}, + {785, 7}, + {785, 7}, + {785, 9}, + {785, 4}, + {785, 6}, + {785, 3}, + {785, 5}, + {803, 1}, + {803, 1}, {1076, 0}, - {1076, 2}, - {871, 0}, - {871, 2}, - {871, 2}, - {1239, 0}, - {1239, 2}, - {1239, 2}, - {1295, 1}, - {876, 1}, - {876, 3}, - {838, 1}, - {838, 4}, - {791, 1}, - {791, 1}, - {790, 6}, - {790, 2}, - {790, 3}, - {847, 0}, - {847, 4}, + {1076, 1}, + {809, 1}, + {809, 2}, + {809, 2}, + {1050, 0}, + {1050, 2}, + {866, 1}, + {866, 1}, + {1260, 1}, + {1260, 1}, + {1190, 1}, + {1190, 1}, + {1185, 0}, + {1185, 1}, + {755, 2}, + {755, 4}, + {755, 4}, + {755, 5}, + {814, 0}, + {814, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1102, 1}, + {1262, 0}, + {1262, 1}, + {1263, 2}, + {1263, 1}, + {851, 1}, {903, 0}, {903, 1}, - {902, 1}, - {902, 2}, - {928, 2}, - {928, 2}, - {928, 2}, - {1207, 0}, - {1207, 2}, - {1207, 3}, - {1207, 3}, - {927, 5}, - {844, 0}, - {844, 1}, - {844, 3}, - {844, 1}, - {844, 3}, - {1043, 1}, - {1043, 2}, - {1044, 0}, - {1044, 1}, - {787, 3}, - {787, 5}, - {787, 7}, - {787, 7}, - {787, 9}, - {787, 4}, - {787, 6}, - {787, 3}, - {787, 5}, - {805, 1}, - {805, 1}, - {1077, 0}, - {1077, 1}, - {811, 1}, - {811, 2}, - {811, 2}, - {1052, 0}, - {1052, 2}, - {868, 1}, - {868, 1}, - {1262, 1}, - {1262, 1}, - {1191, 1}, - {1191, 1}, - {1186, 0}, - {1186, 1}, - {757, 2}, - {757, 4}, - {757, 4}, - {757, 5}, - {816, 0}, - {816, 1}, - {1103, 1}, - {1103, 1}, - {1103, 1}, - {1103, 1}, {1103, 1}, {1103, 1}, - {1103, 1}, - {1103, 1}, - {1103, 1}, - {1264, 0}, - {1264, 1}, + {1261, 1}, + {947, 0}, + {947, 1}, + {873, 0}, + {873, 5}, + {697, 3}, + {697, 3}, + {697, 3}, + {697, 3}, + {872, 0}, + {872, 3}, + {872, 3}, + {872, 4}, + {872, 5}, + {872, 4}, + {872, 5}, + {872, 5}, + {872, 4}, + {1065, 0}, + {1065, 2}, + {751, 1}, + {751, 1}, + {751, 2}, + {751, 2}, + {746, 3}, + {746, 3}, + {745, 4}, + {745, 4}, + {745, 5}, + {745, 2}, + {745, 2}, + {745, 3}, + {744, 1}, + {744, 3}, + {743, 1}, + {743, 1}, {1265, 2}, - {1265, 1}, - {853, 1}, - {905, 0}, - {905, 1}, + {1265, 2}, + {1265, 2}, + {948, 1}, + {981, 9}, + {981, 9}, + {849, 2}, + {849, 4}, + {849, 6}, + {849, 4}, + {849, 4}, + {849, 3}, + {849, 6}, + {849, 6}, + {1106, 3}, + {1105, 6}, {1104, 1}, {1104, 1}, - {1263, 1}, - {949, 0}, - {949, 1}, - {875, 0}, - {875, 5}, - {699, 3}, - {699, 3}, - {699, 3}, - {699, 3}, - {874, 0}, - {874, 3}, - {874, 3}, - {874, 4}, - {874, 5}, - {874, 4}, - {874, 5}, - {874, 5}, - {874, 4}, - {1066, 0}, - {1066, 2}, - {753, 1}, - {753, 1}, - {753, 2}, - {753, 2}, - {748, 3}, - {748, 3}, - {747, 4}, - {747, 4}, - {747, 5}, - {747, 2}, - {747, 2}, - {747, 3}, - {746, 1}, - {746, 3}, - {745, 1}, - {745, 1}, - {1267, 2}, - {1267, 2}, - {1267, 2}, - {950, 1}, - {983, 9}, - {983, 9}, - {851, 2}, - {851, 4}, - {851, 6}, - {851, 4}, - {851, 4}, - {851, 3}, - {851, 6}, - {851, 6}, - {1107, 3}, - {1106, 6}, - {1105, 1}, - {1105, 1}, - {1105, 1}, - {1268, 3}, - {1268, 1}, - {1268, 1}, - {955, 1}, - {955, 3}, - {908, 3}, - {908, 2}, - {908, 2}, - {908, 3}, - {1214, 2}, - {1214, 2}, - {1214, 2}, - {1214, 1}, - {828, 1}, - {828, 1}, - {828, 1}, - {812, 1}, - {812, 1}, - {818, 1}, - {818, 3}, - {889, 1}, - {889, 3}, - {889, 3}, - {962, 3}, - {962, 4}, - {962, 4}, - {962, 4}, - {962, 3}, - {962, 3}, - {962, 2}, - {962, 4}, - {962, 4}, - {962, 2}, - {962, 2}, - {1162, 1}, - {1162, 1}, - {795, 1}, - {795, 1}, - {858, 1}, - {858, 1}, - {1137, 1}, - {1137, 3}, - {717, 1}, - {717, 1}, - {716, 1}, - {700, 1}, - {762, 1}, - {762, 3}, - {762, 2}, - {762, 2}, - {854, 1}, - {854, 3}, - {1081, 1}, - {1081, 4}, - {879, 1}, - {809, 1}, - {809, 1}, - {789, 3}, - {789, 2}, - {947, 1}, - {947, 1}, - {808, 1}, - {808, 1}, - {849, 1}, - {849, 3}, - {964, 3}, - {964, 5}, - {964, 6}, - {964, 4}, - {964, 4}, - {964, 5}, - {964, 5}, - {964, 5}, - {964, 6}, - {964, 4}, - {964, 5}, - {964, 6}, - {964, 4}, - {964, 3}, - {964, 3}, - {964, 4}, - {964, 4}, - {964, 5}, - {964, 5}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 3}, - {964, 4}, - {1145, 2}, - {1145, 2}, - {1145, 3}, - {1145, 3}, - {1200, 1}, - {1200, 3}, - {1039, 5}, - {1063, 1}, - {1063, 3}, - {1110, 3}, - {1110, 4}, - {1110, 4}, - {1110, 5}, - {1110, 4}, - {1110, 5}, - {1110, 4}, - {1110, 4}, - {1110, 6}, - {1110, 4}, - {1110, 8}, - {1110, 2}, - {1110, 5}, - {1110, 3}, - {1110, 3}, - {1110, 2}, - {1110, 5}, - {1110, 2}, - {1110, 2}, - {1110, 4}, - {1271, 2}, - {1271, 2}, - {1271, 4}, - {1274, 0}, - {1274, 1}, - {1273, 1}, - {1273, 3}, - {1109, 1}, - {1109, 1}, + {1104, 1}, + {1266, 3}, + {1266, 1}, + {1266, 1}, + {953, 1}, + {953, 3}, + {906, 3}, + {906, 2}, + {906, 2}, + {906, 3}, + {1213, 2}, + {1213, 2}, + {1213, 2}, + {1213, 1}, + {826, 1}, + {826, 1}, + {826, 1}, + {810, 1}, + {810, 1}, + {816, 1}, + {816, 3}, + {887, 1}, + {887, 3}, + {887, 3}, + {960, 3}, + {960, 4}, + {960, 4}, + {960, 4}, + {960, 3}, + {960, 3}, + {960, 2}, + {960, 4}, + {960, 4}, + {960, 2}, + {960, 2}, + {1161, 1}, + {1161, 1}, + {793, 1}, + {793, 1}, + {856, 1}, + {856, 1}, + {1136, 1}, + {1136, 3}, + {715, 1}, + {715, 1}, + {714, 1}, + {698, 1}, + {760, 1}, + {760, 3}, + {760, 2}, + {760, 2}, + {852, 1}, + {852, 3}, + {1080, 1}, + {1080, 4}, + {877, 1}, + {807, 1}, + {807, 1}, + {787, 3}, + {787, 2}, + {945, 1}, + {945, 1}, + {806, 1}, + {806, 1}, + {847, 1}, + {847, 3}, + {962, 3}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 4}, + {962, 5}, + {962, 5}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 5}, + {962, 6}, + {962, 4}, + {962, 3}, + {962, 3}, + {962, 4}, + {962, 4}, + {962, 5}, + {962, 5}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 3}, + {962, 4}, + {1144, 2}, + {1144, 2}, + {1144, 3}, + {1144, 3}, + {1199, 1}, + {1199, 3}, + {1037, 5}, + {1062, 1}, + {1062, 3}, + {1109, 3}, + {1109, 4}, + {1109, 4}, + {1109, 5}, + {1109, 4}, + {1109, 5}, + {1109, 4}, + {1109, 4}, + {1109, 6}, + {1109, 4}, + {1109, 8}, {1109, 2}, + {1109, 5}, + {1109, 3}, + {1109, 3}, {1109, 2}, + {1109, 5}, {1109, 2}, - {1109, 1}, - {1109, 1}, - {1109, 1}, - {1109, 1}, + {1109, 2}, + {1109, 4}, + {1269, 2}, + {1269, 2}, + {1269, 4}, {1272, 0}, - {1272, 3}, - {1306, 0}, - {1306, 2}, - {1269, 1}, - {1269, 1}, - {1269, 1}, - {793, 1}, - {793, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 3}, - {1275, 3}, - {1275, 3}, - {1275, 3}, - {1275, 5}, - {1275, 4}, - {1275, 5}, - {1275, 1}, - {1275, 1}, - {1275, 2}, - {1275, 2}, - {1275, 2}, - {1275, 1}, - {1275, 2}, - {1275, 2}, - {1275, 2}, - {1275, 2}, - {1275, 2}, - {1275, 2}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 2}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 1}, - {1275, 2}, + {1272, 1}, + {1271, 1}, + {1271, 3}, + {1108, 1}, + {1108, 1}, + {1108, 2}, + {1108, 2}, + {1108, 2}, + {1108, 1}, + {1108, 1}, + {1108, 1}, + {1108, 1}, {1270, 0}, - {1270, 2}, - {1270, 2}, - {925, 0}, - {925, 1}, - {925, 1}, - {1283, 0}, - {1283, 1}, - {1283, 1}, - {1283, 1}, - {1071, 0}, - {1071, 1}, - {829, 0}, - {829, 2}, - {1111, 2}, - {1033, 3}, - {939, 1}, - {939, 3}, - {1195, 1}, - {1195, 1}, - {1195, 3}, - {1195, 1}, - {1195, 2}, - {1195, 3}, - {1195, 1}, - {1225, 0}, - {1225, 1}, - {1225, 1}, - {1225, 1}, - {1225, 1}, - {1225, 1}, - {824, 0}, - {824, 1}, - {824, 1}, - {1126, 0}, - {1126, 1}, - {953, 0}, - {953, 2}, - {1325, 0}, - {1325, 3}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {1116, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {907, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {839, 1}, - {1282, 1}, - {1282, 3}, - {890, 2}, - {984, 1}, - {984, 1}, - {952, 1}, - {952, 1}, - {1124, 1}, - {1124, 3}, - {1293, 0}, - {1293, 3}, - {830, 1}, - {830, 4}, - {830, 4}, - {830, 4}, - {830, 3}, - {830, 4}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 1}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 3}, - {830, 2}, - {830, 2}, - {830, 3}, - {830, 3}, - {830, 5}, - {830, 3}, - {822, 0}, - {822, 1}, - {1118, 1}, - {1118, 1}, - {1001, 0}, - {1001, 1}, - {906, 1}, - {906, 2}, - {906, 3}, - {1242, 0}, - {1242, 1}, - {1132, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {826, 3}, - {1303, 1}, - {1303, 1}, - {1303, 1}, - {1231, 3}, - {1231, 2}, - {1231, 3}, - {1231, 3}, - {1231, 2}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1213, 1}, - {1160, 1}, - {1160, 1}, - {1072, 0}, - {1072, 1}, - {1072, 1}, - {1192, 1}, - {1192, 1}, - {1192, 1}, + {1270, 3}, + {1304, 0}, + {1304, 2}, + {1267, 1}, + {1267, 1}, + {1267, 1}, + {791, 1}, + {791, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 3}, + {1273, 3}, + {1273, 3}, + {1273, 3}, + {1273, 5}, + {1273, 4}, + {1273, 5}, + {1273, 1}, + {1273, 1}, + {1273, 2}, + {1273, 2}, + {1273, 2}, + {1273, 1}, + {1273, 2}, + {1273, 2}, + {1273, 2}, + {1273, 2}, + {1273, 2}, + {1273, 2}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 2}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 1}, + {1273, 2}, + {1268, 0}, + {1268, 2}, + {1268, 2}, + {923, 0}, + {923, 1}, + {923, 1}, + {1281, 0}, + {1281, 1}, + {1281, 1}, + {1281, 1}, + {1070, 0}, + {1070, 1}, + {827, 0}, + {827, 2}, + {1110, 2}, + {1031, 3}, + {937, 1}, + {937, 3}, {1194, 1}, {1194, 1}, + {1194, 3}, {1194, 1}, {1194, 2}, - {1158, 1}, - {1288, 3}, - {1288, 2}, - {1288, 3}, - {1288, 2}, - {1288, 3}, - {1288, 3}, - {1288, 2}, - {1288, 2}, - {1288, 1}, - {1288, 2}, - {1288, 5}, - {1288, 5}, - {1288, 1}, - {1288, 3}, - {1288, 2}, - {888, 1}, - {888, 1}, - {1230, 1}, - {1230, 2}, - {1230, 2}, - {1136, 2}, - {1136, 2}, - {1136, 1}, - {1136, 1}, - {1232, 2}, - {1232, 2}, - {1232, 1}, - {1232, 2}, - {1232, 2}, - {1232, 3}, - {1232, 3}, - {1232, 2}, - {1328, 1}, - {1328, 1}, - {1159, 1}, - {1159, 2}, + {1194, 3}, + {1194, 1}, + {1223, 0}, + {1223, 1}, + {1223, 1}, + {1223, 1}, + {1223, 1}, + {1223, 1}, + {822, 0}, + {822, 1}, + {822, 1}, + {1125, 0}, + {1125, 1}, + {951, 0}, + {951, 2}, + {1323, 0}, + {1323, 3}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {1115, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {905, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {837, 1}, + {1280, 1}, + {1280, 3}, + {888, 2}, + {982, 1}, + {982, 1}, + {950, 1}, + {950, 1}, + {1123, 1}, + {1123, 3}, + {1291, 0}, + {1291, 3}, + {828, 1}, + {828, 4}, + {828, 4}, + {828, 4}, + {828, 3}, + {828, 4}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 1}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 3}, + {828, 2}, + {828, 2}, + {828, 3}, + {828, 3}, + {828, 5}, + {828, 3}, + {820, 0}, + {820, 1}, + {1117, 1}, + {1117, 1}, + {999, 0}, + {999, 1}, + {904, 1}, + {904, 2}, + {904, 3}, + {1240, 0}, + {1240, 1}, + {1131, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {824, 3}, + {1301, 1}, + {1301, 1}, + {1301, 1}, + {1229, 3}, + {1229, 2}, + {1229, 3}, + {1229, 3}, + {1229, 2}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, + {1212, 1}, {1159, 1}, {1159, 1}, - {1159, 2}, - {1300, 1}, - {1300, 2}, - {1300, 1}, - {1300, 1}, - {870, 1}, - {870, 1}, - {870, 1}, - {870, 1}, - {1178, 1}, - {1178, 2}, - {1178, 2}, - {1178, 2}, - {1178, 3}, - {750, 3}, - {774, 0}, - {774, 1}, - {861, 1}, - {861, 1}, - {861, 1}, - {862, 0}, - {862, 2}, - {891, 0}, - {891, 1}, - {891, 1}, - {896, 5}, - {1235, 0}, - {1235, 1}, - {788, 0}, - {788, 2}, - {788, 3}, - {1236, 0}, - {1236, 2}, - {761, 2}, - {761, 1}, - {761, 2}, - {1070, 0}, - {1070, 2}, + {1071, 0}, + {1071, 1}, + {1071, 1}, + {1191, 1}, + {1191, 1}, + {1191, 1}, + {1193, 1}, + {1193, 1}, + {1193, 1}, + {1193, 2}, + {1157, 1}, + {1286, 3}, + {1286, 2}, + {1286, 3}, + {1286, 2}, + {1286, 3}, + {1286, 3}, + {1286, 2}, + {1286, 2}, + {1286, 1}, + {1286, 2}, + {1286, 5}, + {1286, 5}, {1286, 1}, {1286, 3}, - {954, 1}, - {954, 1}, - {954, 1}, - {1130, 1}, - {1130, 3}, - {728, 1}, - {728, 1}, - {1287, 1}, - {1287, 1}, - {1287, 1}, - {771, 1}, - {771, 2}, - {763, 10}, - {763, 8}, + {1286, 2}, + {886, 1}, + {886, 1}, + {1228, 1}, + {1228, 2}, + {1228, 2}, {1135, 2}, - {778, 2}, - {779, 0}, - {779, 1}, - {1333, 0}, - {1333, 1}, - {1002, 7}, - {998, 4}, - {974, 7}, - {974, 9}, - {968, 3}, - {1212, 2}, - {1212, 6}, - {877, 2}, - {909, 1}, - {909, 3}, - {992, 0}, - {992, 2}, - {1172, 1}, - {1172, 2}, - {991, 2}, - {991, 2}, - {991, 2}, - {991, 2}, - {945, 0}, - {945, 1}, + {1135, 2}, + {1135, 1}, + {1135, 1}, + {1230, 2}, + {1230, 2}, + {1230, 1}, + {1230, 2}, + {1230, 2}, + {1230, 3}, + {1230, 3}, + {1230, 2}, + {1326, 1}, + {1326, 1}, + {1158, 1}, + {1158, 2}, + {1158, 1}, + {1158, 1}, + {1158, 2}, + {1298, 1}, + {1298, 2}, + {1298, 1}, + {1298, 1}, + {868, 1}, + {868, 1}, + {868, 1}, + {868, 1}, + {1177, 1}, + {1177, 2}, + {1177, 2}, + {1177, 2}, + {1177, 3}, + {748, 3}, + {772, 0}, + {772, 1}, + {859, 1}, + {859, 1}, + {859, 1}, + {860, 0}, + {860, 2}, + {889, 0}, + {889, 1}, + {889, 1}, + {894, 5}, + {1233, 0}, + {1233, 1}, + {786, 0}, + {786, 2}, + {786, 3}, + {1234, 0}, + {1234, 2}, + {759, 2}, + {759, 1}, + {759, 2}, + {1069, 0}, + {1069, 2}, + {1284, 1}, + {1284, 3}, + {952, 1}, + {952, 1}, + {952, 1}, + {1129, 1}, + {1129, 3}, + {726, 1}, + {726, 1}, + {1285, 1}, + {1285, 1}, + {1285, 1}, + {769, 1}, + {769, 2}, + {761, 10}, + {761, 8}, + {1134, 2}, + {776, 2}, + {777, 0}, + {777, 1}, + {1331, 0}, + {1331, 1}, + {1000, 7}, + {996, 4}, + {972, 7}, + {972, 9}, + {966, 3}, + {1211, 2}, + {1211, 6}, + {875, 2}, + {907, 1}, + {907, 3}, + {990, 0}, + {990, 2}, + {1171, 1}, + {1171, 2}, + {989, 2}, + {989, 2}, + {989, 2}, + {989, 2}, + {943, 0}, + {943, 1}, + {942, 2}, + {942, 2}, + {942, 2}, + {942, 2}, + {1258, 1}, + {1258, 3}, + {1258, 2}, {944, 2}, {944, 2}, {944, 2}, {944, 2}, - {1260, 1}, - {1260, 3}, - {1260, 2}, - {946, 2}, - {946, 2}, - {946, 2}, - {946, 2}, - {1083, 0}, - {1083, 1}, + {1082, 0}, {1082, 1}, - {1082, 2}, - {938, 2}, - {938, 2}, - {938, 1}, - {938, 4}, - {938, 2}, - {938, 2}, - {937, 3}, - {1164, 0}, - {1155, 0}, - {1155, 3}, - {1155, 3}, - {1155, 5}, - {1155, 5}, - {1155, 4}, - {1156, 1}, - {1040, 1}, - {1040, 1}, - {1102, 1}, - {1261, 1}, - {1261, 3}, - {880, 1}, - {880, 1}, - {880, 1}, - {880, 1}, - {880, 1}, - {880, 1}, - {880, 1}, - {880, 1}, - {993, 7}, - {1009, 5}, - {1009, 7}, - {1038, 9}, - {1036, 7}, - {1037, 4}, - {1142, 0}, - {1142, 3}, - {1142, 3}, - {1142, 3}, - {1142, 3}, - {1142, 3}, - {923, 1}, - {923, 2}, - {948, 1}, - {948, 1}, - {948, 1}, - {948, 3}, - {948, 3}, - {1101, 1}, - {1101, 3}, - {941, 1}, - {941, 4}, - {942, 1}, - {942, 2}, - {942, 1}, - {942, 1}, - {942, 2}, - {942, 2}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 2}, - {942, 1}, - {942, 2}, - {942, 1}, - {942, 2}, - {942, 2}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 1}, - {942, 3}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 2}, - {942, 1}, - {942, 1}, - {1064, 0}, - {1064, 1}, - {1064, 1}, - {1064, 1}, - {1087, 1}, - {1087, 3}, - {1087, 3}, - {1087, 3}, - {1087, 1}, - {1100, 7}, - {1099, 4}, - {846, 15}, - {1205, 0}, - {1205, 3}, + {1081, 1}, + {1081, 2}, + {936, 2}, + {936, 2}, + {936, 1}, + {936, 4}, + {936, 2}, + {936, 2}, + {935, 3}, {1163, 0}, - {1163, 3}, - {1057, 0}, - {1057, 1}, - {1031, 0}, - {1031, 2}, - {821, 1}, - {821, 1}, - {1189, 2}, - {1189, 1}, - {1030, 3}, - {1030, 4}, - {1030, 3}, - {1030, 3}, - {840, 1}, - {840, 1}, - {840, 1}, - {931, 0}, - {931, 3}, - {1280, 0}, - {1280, 3}, - {1220, 0}, + {1154, 0}, + {1154, 3}, + {1154, 3}, + {1154, 5}, + {1154, 5}, + {1154, 4}, + {1155, 1}, + {1038, 1}, + {1038, 1}, + {1101, 1}, + {1259, 1}, + {1259, 3}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {878, 1}, + {991, 7}, + {1007, 5}, + {1007, 7}, + {1036, 9}, + {1034, 7}, + {1035, 4}, + {1141, 0}, + {1141, 3}, + {1141, 3}, + {1141, 3}, + {1141, 3}, + {1141, 3}, + {921, 1}, + {921, 2}, + {946, 1}, + {946, 1}, + {946, 1}, + {946, 3}, + {946, 3}, + {1100, 1}, + {1100, 3}, + {939, 1}, + {939, 4}, + {940, 1}, + {940, 2}, + {940, 1}, + {940, 1}, + {940, 2}, + {940, 2}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 2}, + {940, 1}, + {940, 2}, + {940, 1}, + {940, 2}, + {940, 2}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 1}, + {940, 3}, + {940, 2}, + {940, 2}, + {940, 2}, + {940, 2}, + {940, 2}, + {940, 2}, + {940, 2}, + {940, 1}, + {940, 1}, + {1063, 0}, + {1063, 1}, + {1063, 1}, + {1063, 1}, + {1086, 1}, + {1086, 3}, + {1086, 3}, + {1086, 3}, + {1086, 1}, + {1099, 7}, + {1098, 4}, + {844, 15}, + {1204, 0}, + {1204, 3}, + {1162, 0}, + {1162, 3}, + {1055, 0}, + {1055, 1}, + {1029, 0}, + {1029, 2}, + {819, 1}, + {819, 1}, + {1188, 2}, + {1188, 1}, + {1028, 3}, + {1028, 4}, + {1028, 3}, + {1028, 3}, + {838, 1}, + {838, 1}, + {838, 1}, + {929, 0}, + {929, 3}, + {1278, 0}, + {1278, 3}, + {1219, 0}, + {1219, 3}, + {1221, 0}, + {1221, 2}, {1220, 3}, - {1222, 0}, + {1220, 1}, + {1053, 3}, + {1132, 2}, + {1057, 3}, + {1127, 1}, + {1127, 1}, + {1124, 2}, + {1222, 1}, {1222, 2}, - {1221, 3}, - {1221, 1}, - {1055, 3}, - {1133, 2}, - {1058, 3}, - {1128, 1}, - {1128, 1}, - {1125, 2}, - {1224, 1}, - {1224, 2}, - {1224, 1}, - {1224, 2}, - {1294, 1}, - {1294, 3}, - {1051, 2}, - {1051, 3}, - {1051, 3}, - {1050, 1}, - {1050, 2}, - {1056, 3}, - {1013, 5}, - {997, 7}, - {970, 6}, - {999, 6}, - {1174, 0}, - {1174, 1}, - {1266, 1}, - {1266, 2}, - {900, 3}, - {900, 3}, - {900, 3}, - {900, 3}, - {900, 3}, - {900, 1}, - {900, 2}, - {900, 3}, - {900, 1}, - {900, 2}, - {900, 3}, - {900, 1}, - {900, 2}, - {900, 1}, - {900, 1}, - {900, 2}, - {801, 1}, - {801, 2}, - {801, 2}, - {1015, 4}, - {972, 5}, - {1147, 1}, - {1147, 2}, - {971, 1}, - {971, 1}, - {971, 3}, - {971, 3}, - {1042, 8}, - {1229, 0}, - {1229, 2}, - {1228, 0}, - {1228, 3}, - {1253, 0}, - {1253, 2}, - {1252, 0}, - {1252, 2}, - {1023, 1}, - {960, 1}, - {960, 3}, - {899, 2}, - {1085, 5}, - {1085, 6}, - {1085, 9}, - {1085, 10}, - {1085, 4}, + {1222, 1}, + {1222, 2}, + {1292, 1}, + {1292, 3}, + {1049, 2}, + {1049, 3}, + {1049, 3}, + {1048, 1}, + {1048, 2}, + {1054, 3}, + {1011, 5}, + {995, 7}, + {968, 6}, + {997, 6}, + {1173, 0}, + {1173, 1}, + {1264, 1}, + {1264, 2}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 3}, + {898, 1}, + {898, 2}, + {898, 1}, + {898, 1}, + {898, 2}, + {799, 1}, + {799, 2}, + {799, 2}, + {1013, 4}, + {970, 5}, + {1146, 1}, + {1146, 2}, + {969, 1}, + {969, 1}, + {969, 3}, + {969, 3}, + {1040, 8}, + {1227, 0}, + {1227, 2}, + {1226, 0}, + {1226, 3}, + {1251, 0}, + {1251, 2}, + {1250, 0}, + {1250, 2}, + {1021, 1}, + {958, 1}, + {958, 3}, + {897, 2}, + {1084, 5}, + {1084, 6}, + {1084, 9}, + {1084, 10}, + {1084, 4}, } yyXErrors = map[yyXError]string{} - yyParseTab = [4158][]uint16{ + yyParseTab = [4163][]uint16{ // 0 - {1997, 1997, 47: 2487, 69: 2602, 71: 2468, 80: 2498, 145: 2470, 151: 2496, 153: 2467, 166: 2492, 198: 2517, 205: 2614, 208: 2463, 216: 2516, 2483, 2469, 233: 2495, 238: 2473, 241: 2493, 243: 2464, 245: 2499, 263: 2485, 267: 2484, 274: 2497, 276: 2465, 279: 2486, 290: 2478, 462: 2507, 2506, 485: 2610, 491: 2505, 495: 2491, 501: 2515, 514: 2605, 518: 2481, 556: 2490, 559: 2504, 636: 2500, 639: 2613, 643: 2466, 2604, 652: 2461, 659: 2472, 664: 2471, 669: 2514, 676: 2462, 699: 2511, 732: 2474, 741: 2513, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2584, 2583, 2477, 763: 2603, 2475, 768: 2567, 2578, 771: 2594, 781: 2476, 785: 2533, 797: 2608, 810: 2521, 832: 2528, 835: 2531, 841: 2606, 846: 2570, 850: 2575, 2585, 2488, 918: 2540, 922: 2479, 957: 2609, 964: 2519, 966: 2520, 2523, 2524, 970: 2526, 972: 2525, 974: 2522, 976: 2527, 2529, 2530, 980: 2489, 2566, 983: 2536, 993: 2544, 2537, 2538, 2539, 2545, 2543, 2546, 2547, 1002: 2542, 2541, 1005: 2532, 2494, 2480, 2548, 2560, 2549, 2550, 2551, 2553, 2557, 2554, 2558, 2559, 2552, 2556, 2555, 1022: 2518, 1026: 2534, 2535, 2482, 1032: 2562, 2561, 1036: 2564, 2565, 2563, 1041: 2600, 2568, 1049: 2612, 2611, 2569, 1056: 2571, 1058: 2597, 1085: 2572, 2573, 1088: 2574, 1090: 2579, 1093: 2576, 2577, 1096: 2599, 2580, 2607, 2582, 2581, 1106: 2587, 2586, 2590, 1110: 2591, 1112: 2598, 1115: 2588, 2601, 1120: 2589, 1131: 2592, 2593, 2596, 1135: 2595, 1279: 2459, 1282: 2460}, - {2458}, - {2457, 6614}, - {16: 6566, 132: 6563, 162: 6564, 186: 6567, 249: 6565, 479: 4085, 559: 1813, 572: 5888, 837: 6562, 842: 4084}, - {162: 6547, 559: 6546}, + {1997, 1997, 47: 2488, 69: 2603, 71: 2469, 80: 2499, 145: 2471, 151: 2497, 153: 2468, 166: 2493, 199: 2518, 205: 2615, 208: 2464, 216: 2517, 2484, 2470, 233: 2496, 238: 2474, 241: 2494, 243: 2465, 245: 2500, 263: 2486, 268: 2485, 275: 2498, 277: 2466, 280: 2487, 292: 2479, 462: 2508, 2507, 485: 2611, 491: 2506, 493: 2516, 496: 2492, 514: 2606, 518: 2482, 556: 2491, 559: 2505, 634: 2501, 637: 2614, 641: 2467, 2605, 650: 2462, 657: 2473, 662: 2472, 667: 2515, 674: 2463, 697: 2512, 730: 2475, 739: 2514, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2585, 2584, 2478, 761: 2604, 2476, 766: 2568, 2579, 769: 2595, 779: 2477, 783: 2534, 795: 2609, 808: 2522, 830: 2529, 833: 2532, 839: 2607, 844: 2571, 848: 2576, 2586, 2489, 916: 2541, 920: 2480, 955: 2610, 962: 2520, 964: 2521, 2524, 2525, 968: 2527, 970: 2526, 972: 2523, 974: 2528, 2530, 2531, 978: 2490, 2567, 981: 2537, 991: 2545, 2538, 2539, 2540, 2546, 2544, 2547, 2548, 1000: 2543, 2542, 1003: 2533, 2495, 2481, 2549, 2561, 2550, 2551, 2552, 2554, 2558, 2555, 2559, 2560, 2553, 2557, 2556, 1020: 2519, 1024: 2535, 2536, 2483, 1030: 2563, 2562, 1034: 2565, 2566, 2564, 1039: 2601, 2569, 1047: 2613, 2612, 2570, 1054: 2572, 1057: 2598, 1084: 2573, 2574, 1087: 2575, 1089: 2580, 1092: 2577, 2578, 1095: 2600, 2581, 2608, 2583, 2582, 1105: 2588, 2587, 2591, 1109: 2592, 1111: 2599, 1114: 2589, 2602, 1119: 2590, 1130: 2593, 2594, 2597, 1134: 2596, 1277: 2460, 1280: 2461}, + {2459}, + {2458, 6620}, + {16: 6572, 132: 6569, 162: 6570, 187: 6573, 249: 6571, 479: 4086, 559: 1813, 572: 5889, 835: 6568, 840: 4085}, + {162: 6553, 559: 6552}, // 5 - {559: 6540}, - {559: 6535}, - {364: 6516, 480: 6517, 559: 2313, 1277: 6515}, - {332: 6471, 559: 6470}, - {2281, 2281, 351: 6469, 358: 6468}, + {559: 6546}, + {559: 6541}, + {366: 6522, 480: 6523, 559: 2314, 1275: 6521}, + {334: 6477, 559: 6476}, + {2282, 2282, 353: 6475, 360: 6474}, // 10 - {389: 6457}, - {464: 6456}, - {2248, 2248, 70: 5730, 493: 5728, 848: 5729, 990: 6455}, - {16: 2047, 81: 2047, 99: 2047, 132: 6237, 139: 2047, 154: 578, 156: 6159, 160: 5385, 162: 6238, 167: 6239, 186: 6241, 5857, 211: 6229, 497: 6236, 559: 2016, 572: 5888, 632: 6231, 639: 2141, 658: 2047, 666: 6233, 837: 6234, 925: 6240, 934: 5384, 1208: 6230, 1246: 6235, 1276: 6232}, - {16: 6166, 99: 6160, 110: 2016, 132: 6164, 154: 578, 156: 6159, 160: 5385, 162: 6161, 166: 1004, 6162, 186: 6167, 5857, 211: 6155, 277: 6163, 559: 2016, 572: 5888, 639: 6157, 837: 6156, 925: 6165, 934: 6158}, + {389: 6463}, + {464: 6462}, + {2249, 2249, 70: 5731, 494: 5729, 846: 5730, 988: 6461}, + {16: 2047, 81: 2047, 99: 2047, 132: 6238, 139: 2047, 154: 578, 156: 6160, 160: 5386, 162: 6239, 167: 6240, 187: 6242, 5858, 211: 6230, 498: 6237, 559: 2016, 572: 5889, 630: 6232, 637: 2142, 656: 2047, 664: 6234, 835: 6235, 923: 6241, 932: 5385, 1207: 6231, 1244: 6236, 1274: 6233}, + {16: 6167, 99: 6161, 110: 2016, 132: 6165, 154: 578, 156: 6160, 160: 5386, 162: 6162, 166: 1004, 6163, 187: 6168, 5858, 211: 6156, 278: 6164, 559: 2016, 572: 5889, 637: 6158, 835: 6157, 923: 6166, 932: 6159}, // 15 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 6154}, - {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 486: 825, 738: 825, 825, 825, 749: 5192, 853: 5193, 905: 6120}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 6155}, + {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 486: 825, 736: 825, 825, 825, 747: 5193, 851: 5194, 903: 6121}, {2024, 2024}, {2023, 2023}, - {462: 2507, 491: 2505, 559: 2504, 636: 2500, 644: 2604, 699: 3785, 732: 2474, 741: 3784, 2501, 2502, 2503, 2512, 2510, 3786, 3787, 763: 6119, 6117, 781: 6118}, + {462: 2508, 491: 2506, 559: 2505, 634: 2501, 642: 2605, 697: 3786, 730: 2475, 739: 3785, 2502, 2503, 2504, 2513, 2511, 3787, 3788, 761: 6120, 6118, 779: 6119}, // 20 - {71: 2468, 145: 2470, 151: 2496, 153: 2467, 205: 6093, 326: 6092, 462: 2507, 2506, 491: 2505, 495: 2491, 501: 6096, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6094, 732: 2474, 741: 6095, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6102, 6101, 2477, 763: 2603, 2475, 768: 6099, 6100, 771: 6098, 781: 2476, 785: 6097, 797: 6108, 832: 6104, 835: 6105, 846: 6103, 850: 6106, 6107, 907: 6091}, - {2: 1992, 1992, 1992, 1992, 1992, 8: 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 51: 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 462: 1992, 1992, 482: 1992, 491: 1992, 495: 1992, 556: 1992, 559: 1992, 636: 1992, 643: 1992, 1992, 652: 1992, 732: 1992}, - {2: 1991, 1991, 1991, 1991, 1991, 8: 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 51: 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 462: 1991, 1991, 482: 1991, 491: 1991, 495: 1991, 556: 1991, 559: 1991, 636: 1991, 643: 1991, 1991, 652: 1991, 732: 1991}, - {2: 1990, 1990, 1990, 1990, 1990, 8: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 51: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 462: 1990, 1990, 482: 1990, 491: 1990, 495: 1990, 556: 1990, 559: 1990, 636: 1990, 643: 1990, 1990, 652: 1990, 732: 1990}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 6068, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 2507, 2506, 482: 6067, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 643: 6069, 2604, 652: 2620, 3818, 2674, 2675, 2673, 699: 2621, 727: 6065, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 6066}, + {71: 2469, 145: 2471, 151: 2497, 153: 2468, 205: 6094, 328: 6093, 462: 2508, 2507, 491: 2506, 493: 6097, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6095, 730: 2475, 739: 6096, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6103, 6102, 2478, 761: 2604, 2476, 766: 6100, 6101, 769: 6099, 779: 2477, 783: 6098, 795: 6109, 830: 6105, 833: 6106, 844: 6104, 848: 6107, 6108, 905: 6092}, + {2: 1992, 1992, 1992, 1992, 1992, 8: 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 51: 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 1992, 462: 1992, 1992, 482: 1992, 491: 1992, 496: 1992, 556: 1992, 559: 1992, 634: 1992, 641: 1992, 1992, 650: 1992, 730: 1992}, + {2: 1991, 1991, 1991, 1991, 1991, 8: 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 51: 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 1991, 462: 1991, 1991, 482: 1991, 491: 1991, 496: 1991, 556: 1991, 559: 1991, 634: 1991, 641: 1991, 1991, 650: 1991, 730: 1991}, + {2: 1990, 1990, 1990, 1990, 1990, 8: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 51: 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 1990, 462: 1990, 1990, 482: 1990, 491: 1990, 496: 1990, 556: 1990, 559: 1990, 634: 1990, 641: 1990, 1990, 650: 1990, 730: 1990}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 6069, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 2508, 2507, 482: 6068, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 641: 6070, 2605, 650: 2621, 3819, 2675, 2676, 2674, 697: 2622, 725: 6066, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 6067}, // 25 - {559: 5983, 572: 5888, 837: 5982, 979: 6061}, - {559: 5983, 572: 5888, 837: 5982, 979: 5981}, - {132: 5979}, - {132: 5974}, - {132: 5968}, + {559: 5984, 572: 5889, 835: 5983, 977: 6062}, + {559: 5984, 572: 5889, 835: 5983, 977: 5982}, + {132: 5980}, + {132: 5975}, + {132: 5969}, // 30 - {14: 3733, 16: 5822, 28: 5848, 5847, 98: 571, 107: 571, 110: 571, 125: 578, 132: 5811, 138: 578, 156: 5856, 181: 5820, 187: 5857, 191: 578, 199: 5858, 5834, 206: 5843, 571, 239: 5840, 262: 5839, 296: 5853, 301: 5821, 308: 5836, 5851, 311: 5828, 318: 5826, 320: 5842, 324: 5832, 327: 5841, 5815, 5850, 331: 5855, 333: 5824, 342: 5816, 350: 5830, 360: 5819, 5818, 368: 5854, 373: 5849, 5846, 5845, 390: 5837, 394: 5833, 494: 3734, 559: 5814, 637: 3732, 639: 5823, 643: 5852, 664: 5813, 761: 5829, 901: 5844, 925: 5835, 930: 5825, 943: 5838, 1004: 5827, 1071: 5817, 1269: 5831, 1275: 5812}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5800, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5802, 2674, 2675, 2673, 1256: 5801}, - {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 484: 825, 738: 825, 825, 825, 749: 5192, 853: 5193, 905: 5787}, - {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 484: 1027, 738: 5197, 5196, 5195, 825: 5198, 873: 5753}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5748, 2674, 2675, 2673}, + {14: 3734, 16: 5823, 28: 5849, 5848, 98: 571, 107: 571, 110: 571, 125: 578, 132: 5812, 138: 578, 156: 5857, 182: 5821, 188: 5858, 192: 578, 200: 5859, 5835, 206: 5844, 571, 239: 5841, 262: 5840, 298: 5854, 303: 5822, 310: 5837, 5852, 313: 5829, 320: 5827, 322: 5843, 326: 5833, 329: 5842, 5816, 5851, 333: 5856, 335: 5825, 344: 5817, 352: 5831, 362: 5820, 5819, 369: 5855, 374: 5850, 5847, 5846, 390: 5838, 394: 5834, 495: 3735, 559: 5815, 635: 3733, 637: 5824, 641: 5853, 662: 5814, 759: 5830, 899: 5845, 923: 5836, 928: 5826, 941: 5839, 1002: 5828, 1070: 5818, 1267: 5832, 1273: 5813}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5801, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5803, 2675, 2676, 2674, 1254: 5802}, + {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 484: 825, 736: 825, 825, 825, 747: 5193, 851: 5194, 903: 5788}, + {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 484: 1027, 736: 5198, 5197, 5196, 823: 5199, 871: 5754}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5749, 2675, 2676, 2674}, // 35 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5742, 2674, 2675, 2673}, - {166: 5740}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5743, 2675, 2676, 2674}, + {166: 5741}, {166: 1005}, - {1003, 1003, 70: 5730, 493: 5728, 848: 5729, 990: 5727}, + {1003, 1003, 70: 5731, 494: 5729, 846: 5730, 988: 5728}, {994, 994}, // 40 {993, 993}, - {464: 5726}, - {2: 830, 830, 830, 830, 830, 8: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 51: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 5697, 5703, 5704, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 462: 830, 464: 830, 830, 830, 468: 830, 471: 830, 830, 474: 830, 830, 830, 481: 830, 491: 830, 494: 830, 830, 830, 503: 5700, 512: 830, 532: 830, 555: 830, 830, 830, 830, 560: 830, 830, 830, 830, 830, 830, 567: 830, 830, 830, 830, 572: 830, 830, 575: 830, 577: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 638: 830, 641: 3445, 735: 3443, 3444, 738: 5197, 5196, 5195, 749: 5192, 758: 5696, 5699, 5695, 772: 5618, 775: 5693, 825: 5694, 853: 5692, 1103: 5702, 5698, 1264: 5691, 5701}, - {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 5666, 488: 237, 2634, 237, 502: 237, 778: 2635, 5667, 1196: 5665}, - {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5656, 926: 5658, 949: 5657}, + {464: 5727}, + {2: 830, 830, 830, 830, 830, 8: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 51: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 5698, 5704, 5705, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 462: 830, 464: 830, 830, 830, 468: 830, 471: 830, 830, 474: 830, 830, 830, 481: 830, 491: 830, 495: 830, 830, 830, 503: 5701, 512: 830, 534: 830, 555: 830, 830, 830, 830, 560: 830, 830, 830, 830, 830, 830, 567: 830, 830, 830, 830, 572: 830, 830, 575: 830, 577: 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 830, 636: 830, 639: 3446, 733: 3444, 3445, 736: 5198, 5197, 5196, 747: 5193, 756: 5697, 5700, 5696, 770: 5619, 773: 5694, 823: 5695, 851: 5693, 1102: 5703, 5699, 1262: 5692, 5702}, + {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 5667, 488: 237, 2635, 237, 502: 237, 776: 2636, 5668, 1195: 5666}, + {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5657, 924: 5659, 947: 5658}, // 45 - {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2637, 756: 2638, 799: 5652}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5647}, - {564: 3793, 899: 3792, 960: 3791}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5634, 2674, 2675, 2673, 917: 5633, 1143: 5631, 1257: 5632}, - {462: 2507, 2506, 491: 2505, 559: 2504, 636: 2500, 699: 5630, 741: 3778, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 3780, 3779, 3777}, + {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2638, 754: 2639, 797: 5653}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5648}, + {564: 3794, 897: 3793, 958: 3792}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5635, 2675, 2676, 2674, 915: 5634, 1142: 5632, 1255: 5633}, + {462: 2508, 2507, 491: 2506, 559: 2505, 634: 2501, 697: 5631, 739: 3779, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 3781, 3780, 3778}, // 50 {801, 801, 50: 801, 461: 801, 463: 801, 470: 801}, {800, 800, 50: 800, 461: 800, 463: 800, 470: 800}, - {469: 5615, 477: 5616, 5617, 1267: 5614}, - {473, 473, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {469: 5616, 477: 5617, 5618, 1265: 5615}, + {473, 473, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, {469: 789, 477: 789, 789}, // 55 {475, 475, 469: 787, 477: 787, 787}, - {239: 5599, 262: 5598}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 5487, 5482, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 5485, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 5484, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 5488, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 5489, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 5483, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 5490, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 5486, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 468: 5492, 494: 3734, 557: 5496, 577: 5495, 637: 3732, 653: 5493, 2674, 2675, 2673, 761: 5497, 818: 5494, 962: 5498, 1137: 5491}, - {15: 5362, 198: 5367, 206: 5365, 208: 5360, 5366, 266: 5364, 302: 5363, 5368, 306: 5361, 321: 5369, 367: 5370, 574: 5359, 852: 5358}, - {19: 550, 110: 550, 125: 550, 136: 4622, 142: 550, 181: 550, 188: 550, 197: 550, 213: 550, 224: 550, 244: 550, 247: 550, 532: 550, 559: 550, 806: 4621, 824: 5331}, + {239: 5600, 262: 5599}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 5488, 5483, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 5486, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 5485, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 5489, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 5490, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 5484, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 5491, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 5487, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 468: 5493, 495: 3735, 557: 5497, 577: 5496, 635: 3733, 651: 5494, 2675, 2676, 2674, 759: 5498, 816: 5495, 960: 5499, 1136: 5492}, + {15: 5363, 199: 5368, 206: 5366, 208: 5361, 5367, 266: 5365, 304: 5364, 5369, 308: 5362, 323: 5370, 368: 5371, 574: 5360, 850: 5359}, + {20: 550, 110: 550, 125: 550, 136: 4623, 142: 550, 182: 550, 189: 550, 198: 550, 213: 550, 224: 550, 244: 550, 247: 550, 534: 550, 559: 550, 804: 4622, 822: 5332}, // 60 {541, 541}, {540, 540}, @@ -6191,30 +6186,30 @@ var ( {458, 458}, {457, 457}, {434, 434}, - {2: 380, 380, 380, 380, 380, 8: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 51: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 559: 5328, 1242: 5329}, + {2: 380, 380, 380, 380, 380, 8: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 51: 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 380, 559: 5329, 1240: 5330}, // 145 {243, 243, 470: 243}, - {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 462: 825, 479: 825, 568: 825, 738: 825, 825, 825, 749: 5192, 853: 5193, 905: 5194}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 5191}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5035, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 5037, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 5043, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 5039, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 5036, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 5044, 3108, 2841, 3061, 5038, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 5041, 5145, 2755, 2991, 5042, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 5040, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5046, 485: 5069, 556: 5063, 634: 5067, 636: 5052, 639: 5062, 641: 5056, 644: 5065, 652: 5057, 3390, 2674, 2675, 2673, 659: 5061, 664: 5058, 728: 5045, 732: 5060, 789: 5047, 797: 5051, 841: 5066, 852: 5064, 923: 5048, 941: 5049, 5055, 947: 5050, 5053, 956: 5059, 958: 5068, 1101: 5146}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5035, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 5037, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 5043, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 5039, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 5036, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 5044, 3108, 2841, 3061, 5038, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 5041, 2754, 2755, 2991, 5042, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 5040, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5046, 485: 5069, 556: 5063, 634: 5067, 636: 5052, 639: 5062, 641: 5056, 644: 5065, 652: 5057, 3390, 2674, 2675, 2673, 659: 5061, 664: 5058, 728: 5045, 732: 5060, 789: 5047, 797: 5051, 841: 5066, 852: 5064, 923: 5048, 941: 5049, 5055, 947: 5050, 5053, 956: 5059, 958: 5068, 1101: 5054}, + {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 462: 825, 479: 825, 568: 825, 736: 825, 825, 825, 747: 5193, 851: 5194, 903: 5195}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 5192}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5036, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 5038, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 5044, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 5040, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 5037, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 5045, 3109, 2842, 3062, 5039, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 5042, 5146, 2756, 2992, 5043, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 5041, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5047, 485: 5070, 556: 5064, 632: 5068, 634: 5053, 637: 5063, 639: 5057, 642: 5066, 650: 5058, 3391, 2675, 2676, 2674, 657: 5062, 662: 5059, 726: 5046, 730: 5061, 787: 5048, 795: 5052, 839: 5067, 850: 5065, 921: 5049, 939: 5050, 5056, 945: 5051, 5054, 954: 5060, 956: 5069, 1100: 5147}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5036, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 5038, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 5044, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 5040, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 5037, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 5045, 3109, 2842, 3062, 5039, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 5042, 2755, 2756, 2992, 5043, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 5041, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5047, 485: 5070, 556: 5064, 632: 5068, 634: 5053, 637: 5063, 639: 5057, 642: 5066, 650: 5058, 3391, 2675, 2676, 2674, 657: 5062, 662: 5059, 726: 5046, 730: 5061, 787: 5048, 795: 5052, 839: 5067, 850: 5065, 921: 5049, 939: 5050, 5056, 945: 5051, 5054, 954: 5060, 956: 5069, 1100: 5055}, // 150 - {20: 4994, 277: 4995}, - {110: 4981, 559: 4982, 1128: 4993}, - {110: 4981, 559: 4982, 1128: 4980}, - {26: 4976, 143: 4977, 496: 2648, 725: 4975}, - {26: 56, 143: 56, 213: 4974, 496: 56}, + {21: 4995, 278: 4996}, + {110: 4982, 559: 4983, 1127: 4994}, + {110: 4982, 559: 4983, 1127: 4981}, + {26: 4977, 143: 4978, 497: 2649, 721: 4976}, + {26: 56, 143: 56, 213: 4975, 497: 56}, // 155 - {292: 4957}, - {365: 2615}, - {317: 2616, 797: 2617}, - {922: 2619}, - {464: 2618}, + {294: 4958}, + {367: 2616}, + {319: 2617, 795: 2618}, + {920: 2620}, + {464: 2619}, // 160 {1, 1}, - {188: 2632, 462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 643: 2631, 2604, 652: 2620, 699: 2621, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 2630}, - {479: 4085, 559: 1813, 842: 4084}, - {436, 436, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {189: 2633, 462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 641: 2632, 2605, 650: 2621, 697: 2622, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 2631}, + {479: 4086, 559: 1813, 840: 4085}, + {436, 436, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, {438, 438, 469: 787, 477: 787, 787}, // 165 {443, 443}, @@ -6226,653 +6221,653 @@ var ( {437, 437}, {435, 435}, {5, 5}, - {188: 4079, 462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 652: 2620, 699: 2621, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 4078}, - {143: 2633}, + {189: 4080, 462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 650: 2621, 697: 2622, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 4079}, + {143: 2634}, // 175 - {237, 237, 483: 237, 488: 237, 2634, 237, 778: 2635, 2636}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4077}, + {237, 237, 483: 237, 488: 237, 2635, 237, 776: 2636, 2637}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4078}, {236, 236, 50: 236, 461: 236, 463: 236, 469: 236, 236, 477: 236, 236, 482: 236, 236, 236, 236, 488: 236, 490: 236, 502: 236, 504: 236, 236}, - {1265, 1265, 483: 1265, 488: 1265, 490: 2637, 756: 2638, 799: 2639}, - {649: 2662}, + {1265, 1265, 483: 1265, 488: 1265, 490: 2638, 754: 2639, 797: 2640}, + {647: 2663}, // 180 {1264, 1264, 50: 1264, 124: 1264, 461: 1264, 463: 1264, 469: 1264, 1264, 477: 1264, 1264, 482: 1264, 1264, 1264, 1264, 488: 1264}, - {841, 841, 483: 2640, 488: 2641, 757: 2642, 816: 2643}, - {496: 2648, 567: 2650, 725: 2647, 734: 2649, 868: 2657}, - {8: 2644, 257: 2645, 1191: 2646}, + {841, 841, 483: 2641, 488: 2642, 755: 2643, 814: 2644}, + {497: 2649, 567: 2651, 721: 2648, 731: 2650, 866: 2658}, + {8: 2645, 257: 2646, 1190: 2647}, {840, 840, 50: 840, 461: 840, 463: 840, 469: 840, 840, 477: 840, 840, 482: 840, 484: 840, 840}, // 185 {3, 3}, - {496: 849, 513: 849, 564: 849, 567: 849}, - {496: 848, 513: 848, 564: 848, 567: 848}, - {496: 2648, 513: 847, 564: 847, 567: 2650, 725: 2647, 734: 2649, 868: 2651, 1186: 2652}, - {1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 13: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 47: 1932, 49: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 81: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 100: 1932, 103: 1932, 105: 1932, 1932, 108: 1932, 1932, 111: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 164: 1932, 201: 1932, 1932, 461: 1932, 1932, 1932, 467: 1932, 1932, 1932, 1932, 473: 1932, 477: 1932, 1932, 1932, 1932, 482: 1932, 484: 1932, 1932, 491: 1932, 1932, 494: 1932, 1932, 513: 1932, 559: 1932, 564: 1932, 636: 1932, 1932, 639: 1932, 643: 1932}, + {497: 849, 513: 849, 564: 849, 567: 849}, + {497: 848, 513: 848, 564: 848, 567: 848}, + {497: 2649, 513: 847, 564: 847, 567: 2651, 721: 2648, 731: 2650, 866: 2652, 1185: 2653}, + {1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 13: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 47: 1932, 49: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 81: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 100: 1932, 103: 1932, 105: 1932, 1932, 108: 1932, 1932, 111: 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 1932, 164: 1932, 176: 1932, 202: 1932, 461: 1932, 1932, 1932, 467: 1932, 1932, 1932, 1932, 473: 1932, 477: 1932, 1932, 1932, 1932, 482: 1932, 484: 1932, 1932, 491: 1932, 1932, 1932, 495: 1932, 1932, 513: 1932, 559: 1932, 564: 1932, 634: 1932, 1932, 637: 1932, 641: 1932}, // 190 - {1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 13: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 49: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 100: 1930, 103: 1930, 105: 1930, 1930, 108: 1930, 1930, 111: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 126: 1930, 1930, 1930, 1930, 164: 1930, 176: 1930, 180: 1930, 201: 1930, 1930, 461: 1930, 1930, 1930, 467: 1930, 1930, 1930, 1930, 473: 1930, 477: 1930, 1930, 1930, 1930, 482: 1930, 1930, 1930, 1930, 488: 1930, 1930, 491: 1930, 1930, 494: 1930, 1930, 513: 1930, 559: 1930, 564: 1930, 636: 1930, 1930, 639: 1930, 643: 1930, 647: 1930, 1930}, + {1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 13: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 49: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 100: 1930, 103: 1930, 105: 1930, 1930, 108: 1930, 1930, 111: 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 1930, 126: 1930, 1930, 1930, 1930, 164: 1930, 176: 1930, 1930, 181: 1930, 202: 1930, 461: 1930, 1930, 1930, 467: 1930, 1930, 1930, 1930, 473: 1930, 477: 1930, 1930, 1930, 1930, 482: 1930, 1930, 1930, 1930, 488: 1930, 1930, 491: 1930, 1930, 1930, 495: 1930, 1930, 513: 1930, 559: 1930, 564: 1930, 634: 1930, 1930, 637: 1930, 641: 1930, 645: 1930, 1930}, {853, 853, 7: 853, 50: 853, 164: 853, 461: 853, 463: 853, 469: 853, 853, 477: 853, 853, 482: 853, 484: 853, 853, 513: 853, 564: 853}, {852, 852, 7: 852, 50: 852, 164: 852, 461: 852, 463: 852, 469: 852, 852, 477: 852, 852, 482: 852, 484: 852, 852, 513: 852, 564: 852}, {513: 846, 564: 846}, - {513: 2654, 564: 2653, 1262: 2655}, + {513: 2655, 564: 2654, 1260: 2656}, // 195 {150: 851}, {150: 850}, - {150: 2656}, + {150: 2657}, {842, 842, 50: 842, 461: 842, 463: 842, 469: 842, 842, 477: 842, 842, 482: 842, 484: 842, 842}, - {845, 845, 7: 2658, 50: 845, 164: 2659, 461: 845, 463: 845, 469: 845, 845, 477: 845, 845, 482: 845, 484: 845, 845}, + {845, 845, 7: 2659, 50: 845, 164: 2660, 461: 845, 463: 845, 469: 845, 845, 477: 845, 845, 482: 845, 484: 845, 845}, // 200 - {496: 2648, 567: 2650, 725: 2647, 734: 2649, 868: 2661}, - {496: 2648, 567: 2650, 725: 2647, 734: 2649, 868: 2660}, + {497: 2649, 567: 2651, 721: 2648, 731: 2650, 866: 2662}, + {497: 2649, 567: 2651, 721: 2648, 731: 2650, 866: 2661}, {843, 843, 50: 843, 461: 843, 463: 843, 469: 843, 843, 477: 843, 843, 482: 843, 484: 843, 843}, {844, 844, 50: 844, 461: 844, 463: 844, 469: 844, 844, 477: 844, 844, 482: 844, 484: 844, 844}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 2666, 857: 3142, 887: 3141}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 2667, 855: 3143, 885: 3142}, // 205 - {1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 4074, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 463: 1496, 1496, 1496, 1496, 1496, 469: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 482: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 492: 1496, 1496, 497: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 533: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 571: 1496, 642: 1496, 645: 1496, 1496}, - {1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 4071, 1495, 1495, 1495, 1495, 1495, 469: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 482: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 492: 1495, 1495, 497: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 533: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 571: 1495, 642: 1495, 645: 1495, 1495}, - {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 497: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 533: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721, 650: 4069}, - {1272, 1272, 7: 1272, 50: 1272, 124: 1272, 461: 1272, 463: 1272, 469: 1272, 1272, 477: 1272, 1272, 482: 1272, 1272, 1272, 1272, 488: 1272, 490: 1272, 493: 3247, 497: 3245, 3246, 3244, 3242, 504: 1272, 1272, 513: 1272, 516: 1272, 1272, 4068, 4067, 723: 3243, 3241, 1245: 4066}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4065}, + {1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 4075, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 463: 1496, 1496, 1496, 1496, 1496, 469: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 482: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 492: 1496, 1496, 1496, 498: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 535: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 571: 1496, 640: 1496, 643: 1496, 1496}, + {1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 4072, 1495, 1495, 1495, 1495, 1495, 469: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 482: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 492: 1495, 1495, 1495, 498: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 535: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 571: 1495, 640: 1495, 643: 1495, 1495}, + {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 721, 498: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 535: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721, 648: 4070}, + {1272, 1272, 7: 1272, 50: 1272, 124: 1272, 461: 1272, 463: 1272, 469: 1272, 1272, 477: 1272, 1272, 482: 1272, 1272, 1272, 1272, 488: 1272, 490: 1272, 494: 3246, 498: 3244, 3245, 3243, 3241, 504: 1272, 1272, 513: 1272, 516: 1272, 1272, 4069, 4068, 722: 3242, 3240, 1243: 4067}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4066}, // 210 - {462: 4037}, - {1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 463: 1892, 1892, 467: 1892, 469: 1892, 1892, 1892, 1892, 477: 1892, 1892, 1892, 482: 1892, 1892, 1892, 1892, 1892, 4020, 1892, 1892, 1892, 492: 1892, 1892, 497: 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 511: 1892, 513: 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 534: 1892, 1892, 4017, 4015, 4014, 4022, 4016, 4018, 4019, 4021, 1171: 4013, 1215: 4012}, - {1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 463: 1867, 1867, 467: 1867, 469: 1867, 1867, 1867, 1867, 477: 1867, 1867, 1867, 482: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 492: 1867, 1867, 497: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 511: 1867, 513: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 534: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867}, - {1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 463: 1840, 1840, 3984, 3983, 1840, 469: 1840, 1840, 1840, 1840, 474: 3586, 3585, 3591, 1840, 1840, 1840, 482: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 492: 1840, 1840, 497: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 3988, 1840, 3587, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 533: 3987, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 3985, 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584, 3994, 3995, 798: 3986, 1092: 3989, 1157: 3991, 1211: 3990, 1218: 3992, 1258: 3993}, - {1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 3980, 1789, 1789, 1789, 1789, 1789, 469: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 482: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 492: 1789, 1789, 497: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 533: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 571: 1789, 642: 1789, 645: 1789, 1789}, + {462: 4038}, + {1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 463: 1892, 1892, 467: 1892, 469: 1892, 1892, 1892, 1892, 477: 1892, 1892, 1892, 482: 1892, 1892, 1892, 1892, 1892, 4021, 1892, 1892, 1892, 492: 1892, 1892, 1892, 498: 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 511: 1892, 513: 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 1892, 533: 1892, 535: 1892, 4018, 4016, 4015, 4023, 4017, 4019, 4020, 4022, 1170: 4014, 1214: 4013}, + {1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 463: 1867, 1867, 467: 1867, 469: 1867, 1867, 1867, 1867, 477: 1867, 1867, 1867, 482: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 492: 1867, 1867, 1867, 498: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 511: 1867, 513: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 533: 1867, 535: 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867, 1867}, + {1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 463: 1840, 1840, 3985, 3984, 1840, 469: 1840, 1840, 1840, 1840, 474: 3587, 3586, 3592, 1840, 1840, 1840, 482: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 492: 1840, 1840, 1840, 498: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 3989, 1840, 3588, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 3988, 1840, 535: 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 1840, 3986, 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585, 3995, 3996, 796: 3987, 1091: 3990, 1156: 3992, 1210: 3991, 1217: 3993, 1256: 3994}, + {1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 3981, 1789, 1789, 1789, 1789, 1789, 469: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 482: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 492: 1789, 1789, 1789, 498: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 535: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 571: 1789, 640: 1789, 643: 1789, 1789}, // 215 - {1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 657: 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788}, - {1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 657: 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787}, - {1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 657: 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786}, - {1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 657: 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785}, - {1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 657: 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784}, + {1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 655: 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788, 1788}, + {1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 655: 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787, 1787}, + {1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 655: 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786, 1786}, + {1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 655: 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785, 1785}, + {1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 655: 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784, 1784}, // 220 - {1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1204, 1783, 1783, 1783, 1783, 1783, 469: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 482: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 492: 1783, 1783, 497: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 533: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 571: 1783, 642: 1783, 645: 1783, 1783}, - {1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 657: 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782}, - {1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 657: 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781}, - {1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 657: 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780}, - {1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 657: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779}, + {1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1204, 1783, 1783, 1783, 1783, 1783, 469: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 482: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 492: 1783, 1783, 1783, 498: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 535: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 571: 1783, 640: 1783, 643: 1783, 1783}, + {1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 655: 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782, 1782}, + {1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 655: 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781, 1781}, + {1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 655: 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780, 1780}, + {1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 655: 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779, 1779}, // 225 - {1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 657: 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778}, - {1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 657: 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777}, - {1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 657: 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776}, - {1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 657: 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775}, - {1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 657: 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774}, + {1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 655: 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778, 1778}, + {1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 655: 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777, 1777}, + {1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 655: 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776, 1776}, + {1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 655: 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775, 1775}, + {1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 655: 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774, 1774}, // 230 - {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 3975, 1773, 1773, 1773, 1773, 1773, 469: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 482: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 492: 1773, 1773, 497: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 533: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 571: 1773, 642: 1773, 645: 1773, 1773}, - {1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 657: 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772}, - {1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 657: 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771}, - {1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 657: 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770}, - {1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 657: 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769}, + {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 3976, 1773, 1773, 1773, 1773, 1773, 469: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 482: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 492: 1773, 1773, 1773, 498: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 535: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 571: 1773, 640: 1773, 643: 1773, 1773}, + {1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 655: 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772, 1772}, + {1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 655: 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771, 1771}, + {1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 655: 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770, 1770}, + {1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 655: 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769, 1769}, // 235 - {1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 657: 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768}, - {1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 657: 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767}, - {1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 657: 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766}, - {1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 657: 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765}, - {1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 657: 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764}, + {1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 655: 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768, 1768}, + {1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 655: 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767, 1767}, + {1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 655: 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766, 1766}, + {1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 655: 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765, 1765}, + {1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 655: 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764, 1764}, // 240 - {1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 657: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763}, - {1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1203, 1762, 1762, 1762, 1762, 1762, 469: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 482: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 492: 1762, 1762, 497: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 533: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 571: 1762, 642: 1762, 645: 1762, 1762}, - {1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 657: 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761}, - {1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 657: 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760}, - {1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 657: 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759}, + {1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 655: 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763, 1763}, + {1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1203, 1762, 1762, 1762, 1762, 1762, 469: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 482: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 492: 1762, 1762, 1762, 498: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 535: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 571: 1762, 640: 1762, 643: 1762, 1762}, + {1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 655: 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761, 1761}, + {1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 655: 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760, 1760}, + {1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 655: 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759, 1759}, // 245 - {1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 657: 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758}, - {1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 657: 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757}, - {1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 657: 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756}, - {1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 657: 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755}, - {1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 657: 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754}, + {1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 655: 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758, 1758}, + {1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 655: 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757, 1757}, + {1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 655: 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756, 1756}, + {1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 655: 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755, 1755}, + {1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 655: 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754, 1754}, // 250 - {1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 657: 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753}, - {1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 657: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752}, - {1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1200, 1751, 3974, 1751, 1751, 1751, 469: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 482: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 492: 1751, 1751, 497: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 533: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 571: 1751, 642: 1751, 645: 1751, 1751}, - {1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 657: 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750}, - {1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1198, 1749, 1749, 1749, 1749, 1749, 469: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 482: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 492: 1749, 1749, 497: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 533: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 571: 1749, 642: 1749, 645: 1749, 1749}, + {1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 655: 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753, 1753}, + {1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 655: 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752, 1752}, + {1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1200, 1751, 3975, 1751, 1751, 1751, 469: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 482: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 492: 1751, 1751, 1751, 498: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 535: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 571: 1751, 640: 1751, 643: 1751, 1751}, + {1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 655: 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750, 1750}, + {1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1198, 1749, 1749, 1749, 1749, 1749, 469: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 482: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 492: 1749, 1749, 1749, 498: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 535: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 571: 1749, 640: 1749, 643: 1749, 1749}, // 255 - {1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 657: 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748}, - {1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 657: 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747}, - {1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 657: 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746}, - {1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 657: 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745}, - {1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 657: 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744}, + {1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 655: 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748, 1748}, + {1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 655: 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747, 1747}, + {1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 655: 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746, 1746}, + {1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 655: 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745, 1745}, + {1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 655: 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744, 1744}, // 260 - {1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 657: 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743}, - {1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 657: 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742}, - {1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 657: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741}, - {1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 657: 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740}, - {1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 657: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739}, + {1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 655: 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743, 1743}, + {1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 655: 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742, 1742}, + {1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 655: 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741, 1741}, + {1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 655: 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740, 1740}, + {1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 655: 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739, 1739}, // 265 - {1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 657: 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738}, - {1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 657: 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737}, - {1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 657: 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736}, - {1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 657: 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735}, - {1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 657: 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734}, + {1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 655: 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738, 1738}, + {1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 655: 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737, 1737}, + {1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 655: 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736, 1736}, + {1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 655: 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735, 1735}, + {1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 655: 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734, 1734}, // 270 - {1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 657: 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733}, - {1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 657: 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732}, - {1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 657: 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731}, - {1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 657: 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730}, - {1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 657: 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729}, + {1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 655: 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733, 1733}, + {1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 655: 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732, 1732}, + {1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 655: 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731, 1731}, + {1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 655: 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730, 1730}, + {1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 655: 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729, 1729}, // 275 - {1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 657: 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728}, - {1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 657: 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727}, - {1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1194, 1726, 1726, 1726, 1726, 1726, 469: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 482: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 492: 1726, 1726, 497: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 533: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 571: 1726, 642: 1726, 645: 1726, 1726}, - {1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 657: 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725}, - {1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 657: 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724}, + {1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 655: 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728, 1728}, + {1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 655: 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727, 1727}, + {1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1194, 1726, 1726, 1726, 1726, 1726, 469: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 482: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 492: 1726, 1726, 1726, 498: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 535: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 571: 1726, 640: 1726, 643: 1726, 1726}, + {1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 655: 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725, 1725}, + {1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 655: 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724, 1724}, // 280 - {1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 657: 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723}, - {1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 657: 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722}, - {1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 657: 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721}, - {1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1197, 1720, 1720, 1720, 1720, 1720, 469: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 482: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 492: 1720, 1720, 497: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 533: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 571: 1720, 642: 1720, 645: 1720, 1720}, - {1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 657: 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719}, + {1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 655: 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723, 1723}, + {1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 655: 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722, 1722}, + {1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 655: 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721, 1721}, + {1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1197, 1720, 1720, 1720, 1720, 1720, 469: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 482: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 492: 1720, 1720, 1720, 498: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 535: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 571: 1720, 640: 1720, 643: 1720, 1720}, + {1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 655: 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719, 1719}, // 285 - {1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 657: 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718}, - {1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 657: 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717}, - {1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 657: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716}, - {1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 657: 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715}, - {1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 657: 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714}, + {1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 655: 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718, 1718}, + {1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 655: 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717, 1717}, + {1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 655: 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716, 1716}, + {1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 655: 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715, 1715}, + {1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 655: 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714, 1714}, // 290 - {1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 657: 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713}, - {1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 657: 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712}, - {1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 657: 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711}, - {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 3971, 1710, 1710, 1710, 1710, 1710, 469: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 482: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 492: 1710, 1710, 497: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 533: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 571: 1710, 642: 1710, 645: 1710, 1710}, - {1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 657: 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709}, + {1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 655: 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713, 1713}, + {1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 655: 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712, 1712}, + {1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 655: 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711, 1711}, + {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 3972, 1710, 1710, 1710, 1710, 1710, 469: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 482: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 492: 1710, 1710, 1710, 498: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 535: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 571: 1710, 640: 1710, 643: 1710, 1710}, + {1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 655: 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709, 1709}, // 295 - {1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 657: 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708}, - {1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 657: 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707}, - {1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 657: 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706}, - {1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 657: 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705}, - {1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 657: 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704}, + {1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 655: 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708, 1708}, + {1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 655: 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707, 1707}, + {1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 655: 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706, 1706}, + {1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 655: 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705, 1705}, + {1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 655: 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704, 1704}, // 300 - {1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 657: 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703}, - {1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 657: 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702}, - {1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 657: 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701}, - {1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 657: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700}, - {1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 657: 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699}, + {1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 655: 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703, 1703}, + {1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 655: 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702, 1702}, + {1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 655: 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701, 1701}, + {1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 655: 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700, 1700}, + {1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 655: 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699, 1699}, // 305 - {1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 657: 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698}, - {1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 657: 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697}, - {1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 657: 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696}, - {1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 657: 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695}, - {1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 657: 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694}, + {1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 655: 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698, 1698}, + {1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 655: 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697, 1697}, + {1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 655: 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696, 1696}, + {1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 655: 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695, 1695}, + {1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 655: 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694, 1694}, // 310 - {1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 657: 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693}, - {1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 657: 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692}, - {1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 657: 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691}, - {1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 657: 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690}, - {1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 657: 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689}, + {1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 655: 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693, 1693}, + {1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 655: 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692, 1692}, + {1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 655: 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691, 1691}, + {1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 655: 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690, 1690}, + {1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 655: 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689, 1689}, // 315 - {1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 657: 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688}, - {1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 657: 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687}, - {1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 657: 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686}, - {1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1181, 1685, 3970, 1685, 1685, 1685, 469: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 482: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 492: 1685, 1685, 497: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 533: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 571: 1685, 642: 1685, 645: 1685, 1685}, - {1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1180, 1684, 3969, 1684, 1684, 1684, 469: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 482: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 492: 1684, 1684, 497: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 533: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 571: 1684, 642: 1684, 645: 1684, 1684}, + {1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 655: 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688, 1688}, + {1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 655: 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687, 1687}, + {1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 655: 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686, 1686}, + {1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1181, 1685, 3971, 1685, 1685, 1685, 469: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 482: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 492: 1685, 1685, 1685, 498: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 535: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 571: 1685, 640: 1685, 643: 1685, 1685}, + {1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1180, 1684, 3970, 1684, 1684, 1684, 469: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 482: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 492: 1684, 1684, 1684, 498: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 535: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 571: 1684, 640: 1684, 643: 1684, 1684}, // 320 - {1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 657: 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683}, - {1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 657: 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682}, - {1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1179, 1681, 1681, 1681, 1681, 1681, 469: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 482: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 492: 1681, 1681, 497: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 533: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 571: 1681, 642: 1681, 645: 1681, 1681}, - {1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 657: 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680}, - {1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 657: 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679}, + {1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 655: 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683, 1683}, + {1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 655: 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682, 1682}, + {1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1179, 1681, 1681, 1681, 1681, 1681, 469: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 482: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 492: 1681, 1681, 1681, 498: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 535: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 571: 1681, 640: 1681, 643: 1681, 1681}, + {1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 655: 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680, 1680}, + {1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 655: 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679, 1679}, // 325 - {1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 657: 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678}, - {1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 657: 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677}, - {1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1176, 1676, 1676, 1676, 1676, 1676, 469: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 482: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 492: 1676, 1676, 497: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 533: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 571: 1676, 642: 1676, 645: 1676, 1676}, - {1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 657: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675}, - {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1177, 1674, 1674, 1674, 1674, 1674, 469: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 482: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 492: 1674, 1674, 497: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 533: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 571: 1674, 642: 1674, 645: 1674, 1674}, + {1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 655: 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678, 1678}, + {1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 655: 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677, 1677}, + {1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1176, 1676, 1676, 1676, 1676, 1676, 469: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 482: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 492: 1676, 1676, 1676, 498: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 535: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 571: 1676, 640: 1676, 643: 1676, 1676}, + {1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 655: 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675, 1675}, + {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1177, 1674, 1674, 1674, 1674, 1674, 469: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 482: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 492: 1674, 1674, 1674, 498: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 535: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 571: 1674, 640: 1674, 643: 1674, 1674}, // 330 - {1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 3959, 1673, 1673, 1673, 1673, 1673, 469: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 482: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 492: 1673, 1673, 497: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 533: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 571: 1673, 642: 1673, 645: 1673, 1673}, - {1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 657: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672}, - {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 657: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671}, - {1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1178, 1670, 1670, 1670, 1670, 1670, 469: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 482: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 492: 1670, 1670, 497: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 533: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 571: 1670, 642: 1670, 645: 1670, 1670}, - {1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 657: 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669}, + {1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 3960, 1673, 1673, 1673, 1673, 1673, 469: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 482: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 492: 1673, 1673, 1673, 498: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 535: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 571: 1673, 640: 1673, 643: 1673, 1673}, + {1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 655: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672}, + {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 655: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671}, + {1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1178, 1670, 1670, 1670, 1670, 1670, 469: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 482: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 492: 1670, 1670, 1670, 498: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 535: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 571: 1670, 640: 1670, 643: 1670, 1670}, + {1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 655: 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669, 1669}, // 335 - {1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1201, 1668, 1668, 1668, 1668, 1668, 469: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 482: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 492: 1668, 1668, 497: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 533: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 571: 1668, 642: 1668, 645: 1668, 1668}, - {1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 657: 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667}, - {1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 657: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666}, - {1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 657: 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665}, - {1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 657: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664}, + {1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1201, 1668, 1668, 1668, 1668, 1668, 469: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 482: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 492: 1668, 1668, 1668, 498: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 535: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 571: 1668, 640: 1668, 643: 1668, 1668}, + {1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 655: 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667, 1667}, + {1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 655: 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666, 1666}, + {1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 655: 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665, 1665}, + {1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 655: 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664, 1664}, // 340 - {1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 657: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663}, - {1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 657: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662}, - {1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 657: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661}, - {1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 657: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660}, - {1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 657: 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659}, + {1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 655: 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663, 1663}, + {1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 655: 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662, 1662}, + {1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 655: 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661, 1661}, + {1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 655: 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660, 1660}, + {1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 655: 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659, 1659}, // 345 - {1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 657: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658}, - {1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 657: 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657}, - {1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 657: 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656}, - {1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1188, 1655, 1655, 1655, 1655, 1655, 469: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 482: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 492: 1655, 1655, 497: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 533: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 571: 1655, 642: 1655, 645: 1655, 1655}, - {1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 657: 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654}, + {1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 655: 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658, 1658}, + {1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 655: 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657, 1657}, + {1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 655: 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656, 1656}, + {1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1188, 1655, 1655, 1655, 1655, 1655, 469: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 482: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 492: 1655, 1655, 1655, 498: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 535: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 571: 1655, 640: 1655, 643: 1655, 1655}, + {1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 655: 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654, 1654}, // 350 - {1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 657: 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653}, - {1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 657: 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652}, - {1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 657: 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651}, - {1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 657: 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650}, - {1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 657: 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649}, + {1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 655: 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653, 1653}, + {1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 655: 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652, 1652}, + {1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 655: 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651, 1651}, + {1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 655: 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650, 1650}, + {1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 655: 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649, 1649}, // 355 - {1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 657: 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648}, - {1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 657: 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647}, - {1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 657: 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646}, - {1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 657: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645}, - {1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 657: 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644}, + {1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 655: 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648, 1648}, + {1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 655: 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647, 1647}, + {1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 655: 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646, 1646}, + {1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 655: 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645, 1645}, + {1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 655: 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644, 1644}, // 360 - {1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 657: 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643}, - {1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 657: 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642}, - {1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 657: 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641}, - {1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 657: 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640}, - {1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 657: 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639}, + {1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 655: 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643, 1643}, + {1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 655: 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642, 1642}, + {1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 655: 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641, 1641}, + {1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 655: 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640, 1640}, + {1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 655: 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639, 1639}, // 365 - {1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 657: 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638}, - {1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 657: 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637}, - {1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 657: 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636}, - {1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 657: 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635}, - {1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1185, 1634, 1634, 1634, 1634, 1634, 469: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 482: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 492: 1634, 1634, 497: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 533: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 571: 1634, 642: 1634, 645: 1634, 1634}, + {1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 655: 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638, 1638}, + {1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 655: 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637, 1637}, + {1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 655: 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636, 1636}, + {1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 655: 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635, 1635}, + {1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1185, 1634, 1634, 1634, 1634, 1634, 469: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 482: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 492: 1634, 1634, 1634, 498: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 535: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 571: 1634, 640: 1634, 643: 1634, 1634}, // 370 - {1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 657: 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633}, - {1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 657: 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632}, - {1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 657: 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631}, - {1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 657: 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630}, - {1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 657: 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629}, + {1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 655: 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633, 1633}, + {1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 655: 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632, 1632}, + {1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 655: 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631, 1631}, + {1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 655: 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630, 1630}, + {1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 655: 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629, 1629}, // 375 - {1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 657: 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628}, - {1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 657: 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627}, - {1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 657: 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626}, - {1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 657: 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625}, - {1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 657: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624}, + {1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 655: 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628, 1628}, + {1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 655: 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627, 1627}, + {1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 655: 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626, 1626}, + {1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 655: 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625, 1625}, + {1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 655: 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624, 1624}, // 380 - {1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 657: 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623}, - {1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 657: 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622}, - {1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 657: 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621}, - {1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 657: 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620}, - {1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 657: 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619}, + {1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 655: 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623, 1623}, + {1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 655: 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622, 1622}, + {1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 655: 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621, 1621}, + {1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 655: 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620, 1620}, + {1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 655: 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619, 1619}, // 385 - {1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 657: 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618}, - {1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1183, 1617, 1617, 1617, 1617, 1617, 469: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 482: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 492: 1617, 1617, 497: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 533: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 571: 1617, 642: 1617, 645: 1617, 1617}, - {1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1202, 1616, 1616, 1616, 1616, 1616, 469: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 482: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 492: 1616, 1616, 497: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 533: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 571: 1616, 642: 1616, 645: 1616, 1616}, - {1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1190, 1615, 1615, 1615, 1615, 1615, 469: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 482: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 492: 1615, 1615, 497: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 533: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 571: 1615, 642: 1615, 645: 1615, 1615}, - {1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 657: 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614}, + {1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 655: 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618, 1618}, + {1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1183, 1617, 1617, 1617, 1617, 1617, 469: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 482: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 492: 1617, 1617, 1617, 498: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 535: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 571: 1617, 640: 1617, 643: 1617, 1617}, + {1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1202, 1616, 1616, 1616, 1616, 1616, 469: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 482: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 492: 1616, 1616, 1616, 498: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 535: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 571: 1616, 640: 1616, 643: 1616, 1616}, + {1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1190, 1615, 1615, 1615, 1615, 1615, 469: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 482: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 492: 1615, 1615, 1615, 498: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 535: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 571: 1615, 640: 1615, 643: 1615, 1615}, + {1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 655: 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614, 1614}, // 390 - {1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 657: 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613}, - {1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 657: 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612}, - {1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1192, 1611, 1611, 1611, 1611, 1611, 469: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 482: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 492: 1611, 1611, 497: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 533: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 571: 1611, 642: 1611, 645: 1611, 1611}, - {1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1191, 1610, 1610, 1610, 1610, 1610, 469: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 482: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 492: 1610, 1610, 497: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 533: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 571: 1610, 642: 1610, 645: 1610, 1610}, - {1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 657: 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609}, + {1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 655: 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613, 1613}, + {1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 655: 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612, 1612}, + {1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1192, 1611, 1611, 1611, 1611, 1611, 469: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 482: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 492: 1611, 1611, 1611, 498: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 535: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 571: 1611, 640: 1611, 643: 1611, 1611}, + {1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1191, 1610, 1610, 1610, 1610, 1610, 469: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 482: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 492: 1610, 1610, 1610, 498: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 535: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 571: 1610, 640: 1610, 643: 1610, 1610}, + {1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 655: 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609, 1609}, // 395 - {1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 657: 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608}, - {1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 657: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607}, - {1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 657: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606}, - {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1182, 1605, 1605, 1605, 1605, 1605, 469: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 482: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 492: 1605, 1605, 497: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 533: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 571: 1605, 642: 1605, 645: 1605, 1605}, - {1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 657: 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604}, + {1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 655: 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608, 1608}, + {1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 655: 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607, 1607}, + {1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 655: 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606, 1606}, + {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1182, 1605, 1605, 1605, 1605, 1605, 469: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 482: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 492: 1605, 1605, 1605, 498: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 535: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 571: 1605, 640: 1605, 643: 1605, 1605}, + {1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 655: 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604, 1604}, // 400 - {1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 657: 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603}, - {1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 657: 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602}, - {1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 657: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601}, - {1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 657: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600}, - {1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 657: 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599}, + {1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 655: 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603}, + {1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 655: 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602, 1602}, + {1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 655: 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601, 1601}, + {1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 655: 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600, 1600}, + {1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 655: 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599, 1599}, // 405 - {1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 657: 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598}, - {1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 657: 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597}, - {1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 657: 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596}, - {1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 657: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595}, - {1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 657: 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594}, + {1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 655: 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598, 1598}, + {1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 655: 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597, 1597}, + {1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 655: 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596, 1596}, + {1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 655: 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595, 1595}, + {1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 655: 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594, 1594}, // 410 - {1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 657: 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593}, - {1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 657: 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592}, - {1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 657: 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591}, - {1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 657: 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590}, - {1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 657: 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589}, + {1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 655: 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593, 1593}, + {1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 655: 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592, 1592}, + {1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 655: 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591, 1591}, + {1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 655: 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590, 1590}, + {1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 655: 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589, 1589}, // 415 - {1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 657: 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588}, - {1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 657: 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587}, - {1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 657: 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586}, - {1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 657: 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585}, - {1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 657: 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584}, + {1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 655: 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588, 1588}, + {1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 655: 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587, 1587}, + {1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 655: 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586, 1586}, + {1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 655: 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585, 1585}, + {1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 655: 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584, 1584}, // 420 - {1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 657: 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583}, - {1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 657: 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582}, - {1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 657: 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581}, - {1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 657: 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580}, - {1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 657: 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579}, + {1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 655: 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583, 1583}, + {1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 655: 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582, 1582}, + {1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 655: 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581, 1581}, + {1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 655: 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580, 1580}, + {1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 655: 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579, 1579}, // 425 - {1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 657: 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578}, - {1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 657: 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577}, - {1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 657: 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576}, - {1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 657: 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575}, - {1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 657: 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574}, + {1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 655: 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578, 1578}, + {1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 655: 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577, 1577}, + {1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 655: 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576, 1576}, + {1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 655: 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575, 1575}, + {1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 655: 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574, 1574}, // 430 - {1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 657: 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573}, - {1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 657: 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572}, - {1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 657: 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571}, - {1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 657: 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570}, - {1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 657: 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569}, + {1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 655: 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573, 1573}, + {1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 655: 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572, 1572}, + {1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 655: 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571, 1571}, + {1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 655: 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570, 1570}, + {1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 655: 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569, 1569}, // 435 - {1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 657: 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568}, - {1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 657: 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567}, - {1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 657: 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566}, - {1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 657: 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565}, - {1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 657: 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564}, + {1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 655: 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568, 1568}, + {1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 655: 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567, 1567}, + {1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 655: 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566, 1566}, + {1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 655: 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565, 1565}, + {1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 655: 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564, 1564}, // 440 - {1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 657: 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563}, - {1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 657: 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562}, - {1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 657: 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561}, - {1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 657: 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560}, - {1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 657: 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559}, + {1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 655: 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563, 1563}, + {1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 655: 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562, 1562}, + {1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 655: 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561, 1561}, + {1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 655: 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560, 1560}, + {1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 655: 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559, 1559}, // 445 - {1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 657: 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558}, - {1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 657: 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557}, - {1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 657: 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556}, - {1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 657: 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555}, - {1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 657: 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554}, + {1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 655: 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558, 1558}, + {1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 655: 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557, 1557}, + {1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 655: 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556, 1556}, + {1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 655: 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555, 1555}, + {1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 655: 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554, 1554}, // 450 - {1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 657: 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553}, - {1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 657: 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552}, - {1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 657: 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551}, - {1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 657: 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550}, - {1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 657: 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549}, + {1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 655: 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553, 1553}, + {1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 655: 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552, 1552}, + {1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 655: 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551, 1551}, + {1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 655: 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550, 1550}, + {1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 655: 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549, 1549}, // 455 - {1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 657: 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548}, - {1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 657: 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547}, - {1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 657: 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546}, - {1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 657: 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545}, - {1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 657: 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544}, + {1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 655: 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548, 1548}, + {1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 655: 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547, 1547}, + {1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 655: 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546, 1546}, + {1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 655: 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545, 1545}, + {1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 655: 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544, 1544}, // 460 - {1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 657: 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543}, - {1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 657: 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542}, - {1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 657: 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541}, - {1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 657: 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540}, - {1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 657: 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539}, + {1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 655: 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543, 1543}, + {1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 655: 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542, 1542}, + {1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 655: 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541, 1541}, + {1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 655: 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540, 1540}, + {1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 655: 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539, 1539}, // 465 - {1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 657: 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538}, - {1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 657: 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537}, - {1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 657: 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536}, - {1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 657: 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535}, - {1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 657: 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534}, + {1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 655: 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538, 1538}, + {1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 655: 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537, 1537}, + {1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 655: 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536, 1536}, + {1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 655: 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535, 1535}, + {1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 655: 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534, 1534}, // 470 - {1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 657: 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533}, - {1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 657: 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532}, - {1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 657: 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531}, - {1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 657: 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530}, - {1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 657: 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529}, + {1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 655: 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533, 1533}, + {1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 655: 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532, 1532}, + {1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 655: 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531, 1531}, + {1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 655: 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530, 1530}, + {1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 655: 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529, 1529}, // 475 - {1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 657: 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528}, - {1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 657: 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527}, - {1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 657: 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526}, - {1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 657: 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525}, - {1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 657: 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524}, + {1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 655: 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528, 1528}, + {1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 655: 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527, 1527}, + {1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 655: 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526, 1526}, + {1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 655: 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525, 1525}, + {1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 655: 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524, 1524}, // 480 - {1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 657: 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523}, - {1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 657: 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522}, - {1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 657: 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521}, - {1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 657: 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520}, - {1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 657: 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519}, + {1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 655: 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523, 1523}, + {1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 655: 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522, 1522}, + {1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 655: 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521, 1521}, + {1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 655: 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520, 1520}, + {1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 655: 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519, 1519}, // 485 - {1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 657: 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518}, - {1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 657: 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517}, - {1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 657: 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516}, - {1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 657: 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515}, - {1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 657: 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514}, + {1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 655: 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518, 1518}, + {1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 655: 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517, 1517}, + {1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 655: 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516, 1516}, + {1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 655: 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515, 1515}, + {1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 655: 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514, 1514}, // 490 - {1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 657: 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513}, - {1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 657: 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512}, - {1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 657: 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511}, - {1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 657: 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510}, - {1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 657: 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509}, + {1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 655: 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513, 1513}, + {1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 655: 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512, 1512}, + {1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 655: 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511, 1511}, + {1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 655: 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510, 1510}, + {1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 655: 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509, 1509}, // 495 - {1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 657: 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508}, - {1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 657: 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507}, - {1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 657: 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506}, - {1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 657: 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505}, - {1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 657: 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504}, + {1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 655: 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508, 1508}, + {1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 655: 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507, 1507}, + {1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 655: 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506, 1506}, + {1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 655: 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505, 1505}, + {1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 655: 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504, 1504}, // 500 - {1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 657: 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503}, - {1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 657: 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502}, - {1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 657: 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501}, - {1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 657: 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500}, - {1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 657: 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499}, + {1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 655: 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503, 1503}, + {1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 655: 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502, 1502}, + {1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 655: 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501, 1501}, + {1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 655: 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500, 1500}, + {1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 655: 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499, 1499}, // 505 - {1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 657: 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498}, - {1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 657: 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497}, - {1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 3956, 1494, 1494, 1494, 1494, 1494, 469: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 482: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 492: 1494, 1494, 497: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 533: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 571: 1494, 642: 1494, 645: 1494, 1494}, - {1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 3945, 1493, 1493, 1493, 1493, 1493, 469: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 482: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 492: 1493, 1493, 497: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 533: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 571: 1493, 642: 1493, 645: 1493, 1493}, - {1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 657: 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492}, + {1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 655: 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498, 1498}, + {1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 655: 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497, 1497}, + {1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 3957, 1494, 1494, 1494, 1494, 1494, 469: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 482: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 492: 1494, 1494, 1494, 498: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 535: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 571: 1494, 640: 1494, 643: 1494, 1494}, + {1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 3946, 1493, 1493, 1493, 1493, 1493, 469: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 482: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 492: 1493, 1493, 1493, 498: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 535: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 571: 1493, 640: 1493, 643: 1493, 1493}, + {1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 655: 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492, 1492}, // 510 - {1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 657: 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491}, - {1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 657: 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490}, - {1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 657: 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489}, - {1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 657: 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488}, - {1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 657: 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487}, + {1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 655: 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491, 1491}, + {1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 655: 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490, 1490}, + {1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 655: 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489, 1489}, + {1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 655: 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488, 1488}, + {1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 655: 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487, 1487}, // 515 - {1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 657: 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486}, - {1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 657: 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485}, - {1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 657: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484}, - {1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 657: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483}, - {1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 657: 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482}, + {1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 655: 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486, 1486}, + {1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 655: 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485, 1485}, + {1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 655: 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484, 1484}, + {1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 655: 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483, 1483}, + {1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 655: 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482, 1482}, // 520 - {1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 657: 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481}, - {1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 657: 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480}, - {1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 657: 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479}, - {1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 657: 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478}, - {1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 657: 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477}, + {1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 655: 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481, 1481}, + {1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 655: 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480, 1480}, + {1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 655: 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479, 1479}, + {1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 655: 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478, 1478}, + {1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 655: 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477, 1477}, // 525 - {1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 657: 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476}, - {1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 657: 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475}, - {1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 657: 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474}, - {1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 657: 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473}, - {1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 657: 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472}, + {1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 655: 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476, 1476}, + {1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 655: 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475, 1475}, + {1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 655: 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474, 1474}, + {1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 655: 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473, 1473}, + {1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 655: 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472, 1472}, // 530 - {1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 657: 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471}, - {1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 657: 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470}, - {1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 657: 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469}, - {1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 657: 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468}, - {1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 657: 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467}, + {1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 655: 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471, 1471}, + {1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 655: 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470, 1470}, + {1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 655: 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469, 1469}, + {1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 655: 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468, 1468}, + {1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 655: 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467, 1467}, // 535 - {1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 657: 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466}, - {1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 657: 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465}, - {1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 657: 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464}, - {1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 657: 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463}, - {1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 657: 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462}, + {1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 655: 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466, 1466}, + {1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 655: 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465, 1465}, + {1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 655: 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464, 1464}, + {1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 655: 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463, 1463}, + {1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 655: 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462, 1462}, // 540 - {1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 657: 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461}, - {1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 657: 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460}, - {1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 657: 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459}, - {1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 657: 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458}, - {1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 657: 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457}, + {1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 655: 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461, 1461}, + {1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 655: 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460, 1460}, + {1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 655: 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459, 1459}, + {1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 655: 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458, 1458}, + {1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 655: 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457, 1457}, // 545 - {1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 657: 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456}, - {1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 657: 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455}, - {1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 657: 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454}, - {1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 657: 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453}, - {1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 657: 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452}, + {1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 655: 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456, 1456}, + {1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 655: 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455, 1455}, + {1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 655: 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454, 1454}, + {1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 655: 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453, 1453}, + {1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 655: 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452, 1452}, // 550 - {1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 657: 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451}, - {1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 657: 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450}, - {1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 657: 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449}, - {1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 657: 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448}, - {1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 657: 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447}, + {1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 655: 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451, 1451}, + {1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 655: 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450, 1450}, + {1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 655: 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449, 1449}, + {1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 655: 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448, 1448}, + {1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 655: 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447, 1447}, // 555 - {1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 657: 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446}, - {1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 657: 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445}, - {1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 657: 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444}, - {1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 657: 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443}, - {1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 657: 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442}, + {1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 655: 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446, 1446}, + {1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 655: 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445, 1445}, + {1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 655: 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444, 1444}, + {1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 655: 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443, 1443}, + {1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 655: 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442, 1442}, // 560 - {1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 657: 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441}, - {1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 657: 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440}, - {1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 657: 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439}, - {1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 657: 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438}, - {1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 657: 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437}, + {1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 655: 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441, 1441}, + {1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 655: 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440, 1440}, + {1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 655: 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439, 1439}, + {1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 655: 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438, 1438}, + {1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 655: 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437, 1437}, // 565 - {1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 657: 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436}, - {1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 657: 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435}, - {1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 657: 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434}, - {1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 657: 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433}, - {1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 657: 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432}, + {1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 655: 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436, 1436}, + {1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 655: 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435, 1435}, + {1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 655: 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434, 1434}, + {1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 655: 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433, 1433}, + {1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 655: 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432, 1432}, // 570 - {1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 657: 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431}, - {1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 657: 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430}, - {1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 657: 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429}, - {1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 657: 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428}, - {1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 657: 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427}, + {1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 655: 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431, 1431}, + {1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 655: 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430, 1430}, + {1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 655: 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429, 1429}, + {1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 655: 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428, 1428}, + {1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 655: 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427, 1427}, // 575 - {1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 657: 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426}, - {1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 657: 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425}, - {1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 657: 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424}, - {1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 657: 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423}, - {1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 657: 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422}, + {1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 655: 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426, 1426}, + {1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 655: 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425, 1425}, + {1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 655: 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424, 1424}, + {1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 655: 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423, 1423}, + {1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 655: 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422, 1422}, // 580 - {1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 657: 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421}, - {1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 657: 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420}, - {1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 657: 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419}, - {1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 657: 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418}, - {1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 657: 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417}, + {1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 655: 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421, 1421}, + {1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 655: 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420, 1420}, + {1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 655: 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419, 1419}, + {1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 655: 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418, 1418}, + {1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 655: 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417, 1417}, // 585 - {1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 657: 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416}, - {1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 657: 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415}, - {1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 657: 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414}, - {1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 657: 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413}, - {1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 657: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412}, + {1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 655: 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416, 1416}, + {1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 655: 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415, 1415}, + {1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 655: 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414}, + {1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 655: 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413, 1413}, + {1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1121, 1412, 1412, 1412, 1412, 1412, 469: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 482: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 492: 1412, 1412, 1412, 498: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 535: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 571: 1412, 640: 1412, 643: 1412, 1412}, // 590 - {1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 657: 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411}, - {1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 657: 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410}, - {1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 657: 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409}, - {1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 657: 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408}, - {1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 657: 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407}, + {1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 655: 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411, 1411}, + {1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 655: 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410, 1410}, + {1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 655: 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409, 1409}, + {1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 655: 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408, 1408}, + {1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 655: 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407, 1407}, // 595 - {1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 657: 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406}, - {1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 657: 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405}, - {1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 657: 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404}, - {1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 657: 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403}, - {1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 657: 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402}, + {1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 655: 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406, 1406}, + {1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 655: 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405, 1405}, + {1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 655: 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404, 1404}, + {1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 655: 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403, 1403}, + {1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 655: 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402, 1402}, // 600 - {1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 657: 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401}, - {1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 657: 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400}, - {1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 657: 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399}, - {1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 657: 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398}, - {1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 3936, 1397, 1397, 1397, 1397, 1397, 469: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 482: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 492: 1397, 1397, 497: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 533: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 571: 1397, 642: 1397, 645: 1397, 1397}, + {1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 655: 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401, 1401}, + {1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 655: 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400, 1400}, + {1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 655: 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399, 1399}, + {1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 655: 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398, 1398}, + {1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 3937, 1397, 1397, 1397, 1397, 1397, 469: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 482: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 492: 1397, 1397, 1397, 498: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 535: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 571: 1397, 640: 1397, 643: 1397, 1397}, // 605 - {1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 657: 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396}, - {1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 657: 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395}, - {1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 657: 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394}, - {1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 657: 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393}, - {1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 657: 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392}, + {1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 655: 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396, 1396}, + {1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 655: 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395, 1395}, + {1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 655: 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394, 1394}, + {1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 655: 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393, 1393}, + {1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 655: 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392, 1392}, // 610 - {1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 657: 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391}, - {1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 657: 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390}, - {1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 657: 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389}, - {1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 657: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388}, - {1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 657: 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387}, + {1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 655: 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391, 1391}, + {1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 655: 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390, 1390}, + {1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 655: 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389, 1389}, + {1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 655: 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388, 1388}, + {1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 655: 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387, 1387}, // 615 - {1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 657: 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386}, - {1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 657: 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385}, - {1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 657: 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384}, - {1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 657: 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383}, - {1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 657: 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382}, + {1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 655: 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386, 1386}, + {1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 655: 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385, 1385}, + {1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 655: 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384, 1384}, + {1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 655: 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383, 1383}, + {1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 655: 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382, 1382}, // 620 - {1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 657: 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381}, - {1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 657: 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380}, - {1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 657: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379}, - {1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 657: 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378}, - {1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 657: 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377}, + {1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 655: 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381, 1381}, + {1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 655: 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380, 1380}, + {1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1120, 1379, 1379, 1379, 1379, 1379, 469: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 482: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 492: 1379, 1379, 1379, 498: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 535: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 571: 1379, 640: 1379, 643: 1379, 1379}, + {1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 655: 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378, 1378}, + {1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 655: 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377, 1377}, // 625 - {1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 657: 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376}, - {1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 657: 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375}, - {1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 657: 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374}, - {1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 657: 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373}, - {1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 657: 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372}, + {1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 655: 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376, 1376}, + {1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 655: 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375, 1375}, + {1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 655: 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374, 1374}, + {1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 655: 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373, 1373}, + {1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 655: 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372, 1372}, // 630 - {1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 657: 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371}, - {1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 657: 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370}, - {1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 657: 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369}, - {1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 657: 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368}, - {1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 3929, 1367, 1367, 1367, 1367, 1367, 469: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 482: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 492: 1367, 1367, 497: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 533: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 571: 1367, 642: 1367, 645: 1367, 1367}, + {1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 655: 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371}, + {1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 655: 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370}, + {1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 655: 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369, 1369}, + {1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 655: 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368, 1368}, + {1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 3930, 1367, 1367, 1367, 1367, 1367, 469: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 482: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 492: 1367, 1367, 1367, 498: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 535: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 571: 1367, 640: 1367, 643: 1367, 1367}, // 635 - {1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 3922, 1366, 1366, 1366, 1366, 1366, 469: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 482: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 492: 1366, 1366, 497: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 533: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 571: 1366, 642: 1366, 645: 1366, 1366}, - {1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 657: 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365}, - {1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 657: 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364}, - {1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 657: 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363}, - {1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 657: 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362}, + {1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 3923, 1366, 1366, 1366, 1366, 1366, 469: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 482: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 492: 1366, 1366, 1366, 498: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 535: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 571: 1366, 640: 1366, 643: 1366, 1366}, + {1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 655: 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365, 1365}, + {1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 655: 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364, 1364}, + {1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 655: 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363, 1363}, + {1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 655: 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362, 1362}, // 640 - {1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 657: 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361}, - {1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 657: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360}, - {1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 657: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359}, - {1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 657: 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358}, - {1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 657: 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357}, + {1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 655: 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361, 1361}, + {1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 655: 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360, 1360}, + {1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 655: 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359, 1359}, + {1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 655: 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358, 1358}, + {1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 655: 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357, 1357}, // 645 - {1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 657: 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356}, - {1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 657: 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355}, - {1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 657: 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354}, - {1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 657: 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353}, - {1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 657: 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352}, + {1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 655: 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356, 1356}, + {1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 655: 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355, 1355}, + {1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 655: 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354, 1354}, + {1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 655: 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353, 1353}, + {1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 655: 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352, 1352}, // 650 - {1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 657: 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351}, - {1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 657: 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350}, - {1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 657: 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349}, - {1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 657: 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348}, - {1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 3902, 1347, 1347, 1347, 1347, 1347, 469: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 482: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 492: 1347, 1347, 497: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 533: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 571: 1347, 642: 1347, 645: 1347, 1347}, + {1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 655: 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351, 1351}, + {1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 655: 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350, 1350}, + {1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 655: 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349, 1349}, + {1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 655: 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348, 1348}, + {1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 3903, 1347, 1347, 1347, 1347, 1347, 469: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 482: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 492: 1347, 1347, 1347, 498: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 535: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 571: 1347, 640: 1347, 643: 1347, 1347}, // 655 - {1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 3894, 1346, 1346, 1346, 1346, 1346, 469: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 482: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 492: 1346, 1346, 497: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 533: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 571: 1346, 642: 1346, 645: 1346, 1346}, - {1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 657: 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345}, - {1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 657: 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344}, - {1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 657: 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343}, - {1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 657: 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342}, + {1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 3895, 1346, 1346, 1346, 1346, 1346, 469: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 482: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 492: 1346, 1346, 1346, 498: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 535: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 571: 1346, 640: 1346, 643: 1346, 1346}, + {1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 655: 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345, 1345}, + {1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 655: 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344, 1344}, + {1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 655: 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343, 1343}, + {1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 655: 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342, 1342}, // 660 - {1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 657: 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341}, - {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 657: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340}, - {1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 657: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339}, - {1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 657: 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338}, - {1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 657: 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337}, + {1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 655: 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341, 1341}, + {1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 655: 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340, 1340}, + {1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 655: 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339, 1339}, + {1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 655: 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338, 1338}, + {1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 655: 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337, 1337}, // 665 - {1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 657: 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336}, - {1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 657: 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335}, - {1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 657: 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334}, - {1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 657: 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333}, - {1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 657: 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332}, + {1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 655: 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336, 1336}, + {1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 655: 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335, 1335}, + {1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 655: 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334, 1334}, + {1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 655: 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333, 1333}, + {1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 655: 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332, 1332}, // 670 - {1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 657: 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331}, - {1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 657: 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330}, - {1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 463: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 492: 1292, 1292, 497: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 533: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 566: 1292, 571: 1292, 574: 1292, 576: 1292, 632: 1292, 1292, 1292, 1292}, - {1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 463: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 492: 1291, 1291, 497: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 533: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 566: 1291, 571: 1291, 574: 1291, 576: 1291, 632: 1291, 1291, 1291, 1291}, - {1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 463: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 492: 1290, 1290, 497: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 533: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 566: 1290, 571: 1290, 574: 1290, 576: 1290, 632: 1290, 1290, 1290, 1290}, + {1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 655: 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331, 1331}, + {1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 655: 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330, 1330}, + {1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 463: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 492: 1292, 1292, 1292, 498: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 535: 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 1292, 566: 1292, 571: 1292, 574: 1292, 576: 1292, 630: 1292, 1292, 1292, 1292}, + {1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 463: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 492: 1291, 1291, 1291, 498: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 535: 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 1291, 566: 1291, 571: 1291, 574: 1291, 576: 1291, 630: 1291, 1291, 1291, 1291}, + {1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 463: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 492: 1290, 1290, 1290, 498: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 535: 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 1290, 566: 1290, 571: 1290, 574: 1290, 576: 1290, 630: 1290, 1290, 1290, 1290}, // 675 - {1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 463: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 492: 1289, 1289, 497: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 533: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 566: 1289, 571: 1289, 574: 1289, 576: 1289, 632: 1289, 1289, 1289, 1289}, - {1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 463: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 492: 1288, 1288, 497: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 533: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 566: 1288, 571: 1288, 574: 1288, 576: 1288, 632: 1288, 1288, 1288, 1288}, - {1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 463: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 492: 1287, 1287, 497: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 533: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 566: 1287, 571: 1287, 574: 1287, 576: 1287, 632: 1287, 1287, 1287, 1287}, - {1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 463: 1286, 3893, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 492: 1286, 1286, 497: 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 533: 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 566: 1286, 571: 1286, 574: 1286, 576: 1286, 632: 1286, 1286, 1286, 1286}, - {464: 3890, 565: 3891, 569: 3892}, + {1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 463: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 492: 1289, 1289, 1289, 498: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 535: 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 1289, 566: 1289, 571: 1289, 574: 1289, 576: 1289, 630: 1289, 1289, 1289, 1289}, + {1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 463: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 492: 1288, 1288, 1288, 498: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 535: 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 1288, 566: 1288, 571: 1288, 574: 1288, 576: 1288, 630: 1288, 1288, 1288, 1288}, + {1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 463: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 492: 1287, 1287, 1287, 498: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 535: 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 1287, 566: 1287, 571: 1287, 574: 1287, 576: 1287, 630: 1287, 1287, 1287, 1287}, + {1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 463: 1286, 3894, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 492: 1286, 1286, 1286, 498: 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 535: 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 1286, 566: 1286, 571: 1286, 574: 1286, 576: 1286, 630: 1286, 1286, 1286, 1286}, + {464: 3891, 565: 3892, 569: 3893}, // 680 - {1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 463: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 492: 1284, 1284, 497: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 533: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 566: 1284, 571: 1284, 574: 1284, 576: 1284, 632: 1284, 1284, 1284, 1284}, - {1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 463: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 492: 1283, 1283, 497: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 533: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 566: 1283, 571: 1283, 574: 1283, 576: 1283, 632: 1283, 1283, 1283, 1283}, - {1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 463: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 492: 1280, 1280, 497: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 533: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 566: 1280, 571: 1280, 574: 1280, 576: 1280, 632: 1280, 1280, 1280, 1280}, - {1275, 1275, 7: 3311, 50: 1275, 124: 1275, 461: 1275, 463: 1275, 469: 1275, 1275, 477: 1275, 1275, 482: 1275, 1275, 1275, 1275, 488: 1275}, + {1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 463: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 492: 1284, 1284, 1284, 498: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 535: 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 1284, 566: 1284, 571: 1284, 574: 1284, 576: 1284, 630: 1284, 1284, 1284, 1284}, + {1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 463: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 492: 1283, 1283, 1283, 498: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 535: 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 1283, 566: 1283, 571: 1283, 574: 1283, 576: 1283, 630: 1283, 1283, 1283, 1283}, + {1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 463: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 492: 1280, 1280, 1280, 498: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 535: 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 1280, 566: 1280, 571: 1280, 574: 1280, 576: 1280, 630: 1280, 1280, 1280, 1280}, + {1275, 1275, 7: 3312, 50: 1275, 124: 1275, 461: 1275, 463: 1275, 469: 1275, 1275, 477: 1275, 1275, 482: 1275, 1275, 1275, 1275, 488: 1275}, {1274, 1274, 7: 1274, 50: 1274, 124: 1274, 461: 1274, 463: 1274, 469: 1274, 1274, 477: 1274, 1274, 482: 1274, 1274, 1274, 1274, 488: 1274, 490: 1274, 504: 1274, 1274, 513: 1274, 516: 1274, 1274}, // 685 - {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 463: 1249, 1249, 1249, 1249, 1249, 469: 1249, 1249, 1249, 1249, 3251, 1249, 1249, 1249, 1249, 1249, 1249, 482: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 492: 1249, 1249, 497: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 533: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 571: 3252}, - {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 463: 1248, 1248, 1248, 1248, 1248, 469: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 482: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 492: 1248, 1248, 497: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 533: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 571: 1248, 642: 3885, 645: 1248, 1248}, - {1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 463: 1245, 1245, 1245, 1245, 1245, 469: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 482: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 492: 1245, 1245, 497: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 533: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 571: 1245, 645: 3881, 3882}, - {1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 463: 1244, 1244, 1244, 1244, 1244, 469: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 482: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 492: 1244, 1244, 497: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 533: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 571: 1244}, - {1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 463: 1243, 1243, 1243, 1243, 1243, 469: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 482: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 492: 1243, 1243, 497: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 533: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 571: 1243}, + {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 463: 1249, 1249, 1249, 1249, 1249, 469: 1249, 1249, 1249, 1249, 3250, 1249, 1249, 1249, 1249, 1249, 1249, 482: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 492: 1249, 1249, 1249, 498: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 535: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 571: 3251}, + {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 463: 1248, 1248, 1248, 1248, 1248, 469: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 482: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 492: 1248, 1248, 1248, 498: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 535: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 571: 1248, 640: 3886, 643: 1248, 1248}, + {1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 463: 1245, 1245, 1245, 1245, 1245, 469: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 482: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 492: 1245, 1245, 1245, 498: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 535: 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 1245, 571: 1245, 643: 3882, 3883}, + {1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 463: 1244, 1244, 1244, 1244, 1244, 469: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 482: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 492: 1244, 1244, 1244, 498: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 535: 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 1244, 571: 1244}, + {1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 463: 1243, 1243, 1243, 1243, 1243, 469: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 482: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 492: 1243, 1243, 1243, 498: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 535: 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 1243, 571: 1243}, // 690 - {1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 463: 1242, 1242, 1242, 1242, 1242, 469: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 482: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 492: 1242, 1242, 497: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 533: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 571: 1242}, - {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 463: 1240, 1240, 1240, 1240, 1240, 469: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 482: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 492: 1240, 1240, 497: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 533: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 571: 1240}, - {1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 463: 1239, 1239, 1239, 1239, 1239, 469: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 482: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 492: 1239, 1239, 497: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 533: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 571: 1239}, - {1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 463: 1238, 1238, 1238, 1238, 1238, 469: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 482: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 492: 1238, 1238, 497: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 533: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 571: 1238}, - {1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 463: 1237, 1237, 1237, 1237, 1237, 469: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 482: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 492: 1237, 1237, 497: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 533: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 571: 1237}, + {1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 463: 1242, 1242, 1242, 1242, 1242, 469: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 482: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 492: 1242, 1242, 1242, 498: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 535: 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 1242, 571: 1242}, + {1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 463: 1240, 1240, 1240, 1240, 1240, 469: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 482: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 492: 1240, 1240, 1240, 498: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 535: 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 1240, 571: 1240}, + {1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 463: 1239, 1239, 1239, 1239, 1239, 469: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 482: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 492: 1239, 1239, 1239, 498: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 535: 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 1239, 571: 1239}, + {1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 463: 1238, 1238, 1238, 1238, 1238, 469: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 482: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 492: 1238, 1238, 1238, 498: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 535: 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 1238, 571: 1238}, + {1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 463: 1237, 1237, 1237, 1237, 1237, 469: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 482: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 492: 1237, 1237, 1237, 498: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 535: 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 1237, 571: 1237}, // 695 - {1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 463: 1236, 1236, 1236, 1236, 1236, 469: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 482: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 492: 1236, 1236, 497: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 533: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 571: 1236}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3880, 3153, 3236, 3152, 3149}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3879, 3153, 3236, 3152, 3149}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3878, 3153, 3236, 3152, 3149}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3877, 3153, 3236, 3152, 3149}, + {1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 463: 1236, 1236, 1236, 1236, 1236, 469: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 482: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 492: 1236, 1236, 1236, 498: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 535: 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 1236, 571: 1236}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3881, 3154, 3235, 3153, 3150}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3880, 3154, 3235, 3153, 3150}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3879, 3154, 3235, 3153, 3150}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3878, 3154, 3235, 3153, 3150}, // 700 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3876, 3153, 3236, 3152, 3149}, - {1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 463: 1229, 1229, 1229, 1229, 1229, 469: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 482: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 492: 1229, 1229, 497: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 533: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 571: 1229}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 2506, 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3776, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 2504, 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 636: 2500, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3775, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3773, 741: 3778, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 3780, 3779, 3777, 765: 3774}, - {462: 3768}, - {462: 2507, 699: 3767}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3877, 3154, 3235, 3153, 3150}, + {1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 463: 1229, 1229, 1229, 1229, 1229, 469: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 482: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 492: 1229, 1229, 1229, 498: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 535: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 571: 1229}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 2507, 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3777, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 2505, 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 634: 2501, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3776, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3774, 739: 3779, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 3781, 3780, 3778, 763: 3775}, + {462: 3769}, + {462: 2508, 697: 3768}, // 705 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3764, 2674, 2675, 2673}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3763, 3153, 3236, 3152, 3149}, - {462: 3758}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 531: 1050, 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3745, 1185: 3746}, - {462: 3687}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3765, 2675, 2676, 2674}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3764, 3154, 3235, 3153, 3150}, + {462: 3759}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 531: 1050, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3746, 1184: 3747}, + {462: 3688}, // 710 - {462: 3684}, - {462: 3676}, + {462: 3685}, + {462: 3677}, {462: 1199}, {462: 1196}, {462: 1195}, @@ -6883,653 +6878,653 @@ var ( {462: 1186}, {462: 1184}, // 720 - {1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 469: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 482: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 492: 1173, 1173, 497: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 533: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 571: 1173}, - {1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 469: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 482: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 492: 1172, 1172, 497: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 533: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 571: 1172}, - {1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 469: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 482: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 492: 1171, 1171, 497: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 533: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 571: 1171}, - {1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 469: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 482: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 492: 1170, 1170, 497: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 533: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 571: 1170}, - {1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 469: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 482: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 492: 1169, 1169, 497: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 533: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 571: 1169}, + {1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 469: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 482: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 492: 1173, 1173, 1173, 498: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 535: 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 1173, 571: 1173}, + {1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 469: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 482: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 492: 1172, 1172, 1172, 498: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 535: 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 1172, 571: 1172}, + {1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 469: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 482: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 492: 1171, 1171, 1171, 498: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 535: 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 1171, 571: 1171}, + {1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 469: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 482: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 492: 1170, 1170, 1170, 498: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 535: 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 1170, 571: 1170}, + {1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 469: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 482: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 492: 1169, 1169, 1169, 498: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 535: 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 1169, 571: 1169}, // 725 - {1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 469: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 482: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 492: 1168, 1168, 497: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 533: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 571: 1168}, - {1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 469: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 482: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 492: 1167, 1167, 497: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 533: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 571: 1167}, - {1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 469: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 482: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 492: 1166, 1166, 497: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 533: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 571: 1166}, - {1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 469: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 482: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 492: 1165, 1165, 497: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 533: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 571: 1165}, - {1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 469: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 482: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 492: 1164, 1164, 497: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 533: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 571: 1164}, + {1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 469: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 482: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 492: 1168, 1168, 1168, 498: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 535: 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 1168, 571: 1168}, + {1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 469: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 482: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 492: 1167, 1167, 1167, 498: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 535: 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 1167, 571: 1167}, + {1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 469: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 482: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 492: 1166, 1166, 1166, 498: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 535: 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 1166, 571: 1166}, + {1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 469: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 482: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 492: 1165, 1165, 1165, 498: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 535: 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 1165, 571: 1165}, + {1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 469: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 482: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 492: 1164, 1164, 1164, 498: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 535: 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 1164, 571: 1164}, // 730 - {462: 3673}, - {462: 3670}, - {1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 3667, 1175, 1175, 1175, 1175, 1175, 469: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 482: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 492: 1175, 1175, 497: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 533: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 571: 1175, 1073: 3668}, - {462: 3665}, - {1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 3661, 1082, 1082, 1082, 1082, 1082, 469: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 482: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 492: 1082, 1082, 497: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 533: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 571: 1082, 1198: 3660}, + {462: 3674}, + {462: 3671}, + {1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 3668, 1175, 1175, 1175, 1175, 1175, 469: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 482: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 492: 1175, 1175, 1175, 498: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 535: 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 1175, 571: 1175, 1072: 3669}, + {462: 3666}, + {1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 3662, 1082, 1082, 1082, 1082, 1082, 469: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 482: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 492: 1082, 1082, 1082, 498: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 535: 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 1082, 571: 1082, 1197: 3661}, // 735 - {462: 3652}, - {462: 3648}, - {462: 3643}, - {462: 3640}, - {462: 3635}, + {462: 3653}, + {462: 3649}, + {462: 3644}, + {462: 3641}, + {462: 3636}, // 740 - {462: 3626}, - {462: 3619}, - {462: 3614}, - {462: 3579}, - {462: 3565}, + {462: 3627}, + {462: 3620}, + {462: 3615}, + {462: 3580}, + {462: 3566}, // 745 - {462: 3548}, - {1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 463: 1129, 1129, 1129, 1129, 1129, 469: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 482: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 492: 1129, 1129, 497: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 533: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 571: 1129}, - {462: 3541}, + {462: 3549}, + {1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 463: 1129, 1129, 1129, 1129, 1129, 469: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 482: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 492: 1129, 1129, 1129, 498: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 535: 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 1129, 571: 1129}, + {462: 3542}, {462: 1123}, {462: 1122}, // 750 - {462: 1121}, - {462: 1120}, - {1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 463: 1114, 1114, 1114, 1114, 1114, 469: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 482: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 492: 1114, 1114, 497: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 533: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 571: 1114}, - {462: 3538}, - {462: 3535}, + {1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 463: 1114, 1114, 1114, 1114, 1114, 469: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 482: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 492: 1114, 1114, 1114, 498: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 535: 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 1114, 571: 1114}, + {462: 3539}, + {462: 3536}, + {462: 3528}, + {462: 3520}, // 755 - {462: 3527}, - {462: 3519}, - {462: 3511}, - {462: 3497}, - {462: 3485}, + {462: 3512}, + {462: 3498}, + {462: 3486}, + {462: 3481}, + {462: 3476}, // 760 - {462: 3480}, - {462: 3475}, - {462: 3470}, - {462: 3465}, - {462: 3460}, + {462: 3471}, + {462: 3466}, + {462: 3461}, + {462: 3456}, + {462: 3443}, // 765 - {462: 3455}, - {462: 3442}, - {462: 3439}, - {462: 3436}, - {462: 3433}, + {462: 3440}, + {462: 3437}, + {462: 3434}, + {462: 3431}, + {462: 3428}, // 770 - {462: 3430}, - {462: 3427}, - {462: 3423}, - {462: 3417}, - {462: 3404}, + {462: 3424}, + {462: 3418}, + {462: 3405}, + {462: 3400}, + {462: 3395}, // 775 - {462: 3399}, - {462: 3394}, - {462: 3239}, - {724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 463: 724, 724, 724, 724, 724, 469: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 482: 724, 724, 724, 724, 724, 724, 724, 724, 724, 492: 724, 724, 497: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 533: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 571: 724}, - {723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 463: 723, 723, 723, 723, 723, 469: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 482: 723, 723, 723, 723, 723, 723, 723, 723, 723, 492: 723, 723, 497: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 533: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 571: 723}, + {462: 3238}, + {724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 463: 724, 724, 724, 724, 724, 469: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 482: 724, 724, 724, 724, 724, 724, 724, 724, 724, 492: 724, 724, 724, 498: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 535: 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 724, 571: 724}, + {723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 463: 723, 723, 723, 723, 723, 469: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 482: 723, 723, 723, 723, 723, 723, 723, 723, 723, 492: 723, 723, 723, 498: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 535: 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 723, 571: 723}, + {722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 463: 722, 722, 722, 722, 722, 469: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 482: 722, 722, 722, 722, 722, 722, 722, 722, 722, 492: 722, 722, 722, 498: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 535: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 571: 722}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3239}, // 780 - {722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 463: 722, 722, 722, 722, 722, 469: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 482: 722, 722, 722, 722, 722, 722, 722, 722, 722, 492: 722, 722, 497: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 533: 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 722, 571: 722}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3240}, - {7: 3248, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3393}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3392}, + {7: 3247, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3394}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3393}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3392}, + {2: 1884, 1884, 1884, 1884, 1884, 8: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 51: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 462: 1884, 464: 1884, 1884, 1884, 468: 1884, 471: 1884, 1884, 474: 1884, 1884, 1884, 481: 1884, 491: 1884, 495: 1884, 1884, 1884, 534: 1884, 555: 1884, 1884, 1884, 1884, 560: 1884, 1884, 1884, 1884, 1884, 1884, 567: 1884, 1884, 1884, 1884, 572: 1884, 1884, 575: 1884, 577: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 636: 1884}, // 785 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3391}, - {2: 1884, 1884, 1884, 1884, 1884, 8: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 51: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 462: 1884, 464: 1884, 1884, 1884, 468: 1884, 471: 1884, 1884, 474: 1884, 1884, 1884, 481: 1884, 491: 1884, 494: 1884, 1884, 1884, 532: 1884, 555: 1884, 1884, 1884, 1884, 560: 1884, 1884, 1884, 1884, 1884, 1884, 567: 1884, 1884, 1884, 1884, 572: 1884, 1884, 575: 1884, 577: 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 1884, 638: 1884}, - {2: 1883, 1883, 1883, 1883, 1883, 8: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 51: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 462: 1883, 464: 1883, 1883, 1883, 468: 1883, 471: 1883, 1883, 474: 1883, 1883, 1883, 481: 1883, 491: 1883, 494: 1883, 1883, 1883, 532: 1883, 555: 1883, 1883, 1883, 1883, 560: 1883, 1883, 1883, 1883, 1883, 1883, 567: 1883, 1883, 1883, 1883, 572: 1883, 1883, 575: 1883, 577: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 638: 1883}, - {2: 1882, 1882, 1882, 1882, 1882, 8: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 51: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 462: 1882, 464: 1882, 1882, 1882, 468: 1882, 471: 1882, 1882, 474: 1882, 1882, 1882, 481: 1882, 491: 1882, 494: 1882, 1882, 1882, 532: 1882, 555: 1882, 1882, 1882, 1882, 560: 1882, 1882, 1882, 1882, 1882, 1882, 567: 1882, 1882, 1882, 1882, 572: 1882, 1882, 575: 1882, 577: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 638: 1882}, - {2: 1881, 1881, 1881, 1881, 1881, 8: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 51: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 462: 1881, 464: 1881, 1881, 1881, 468: 1881, 471: 1881, 1881, 474: 1881, 1881, 1881, 481: 1881, 491: 1881, 494: 1881, 1881, 1881, 532: 1881, 555: 1881, 1881, 1881, 1881, 560: 1881, 1881, 1881, 1881, 1881, 1881, 567: 1881, 1881, 1881, 1881, 572: 1881, 1881, 575: 1881, 577: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 638: 1881}, + {2: 1883, 1883, 1883, 1883, 1883, 8: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 51: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 462: 1883, 464: 1883, 1883, 1883, 468: 1883, 471: 1883, 1883, 474: 1883, 1883, 1883, 481: 1883, 491: 1883, 495: 1883, 1883, 1883, 534: 1883, 555: 1883, 1883, 1883, 1883, 560: 1883, 1883, 1883, 1883, 1883, 1883, 567: 1883, 1883, 1883, 1883, 572: 1883, 1883, 575: 1883, 577: 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 1883, 636: 1883}, + {2: 1882, 1882, 1882, 1882, 1882, 8: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 51: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 462: 1882, 464: 1882, 1882, 1882, 468: 1882, 471: 1882, 1882, 474: 1882, 1882, 1882, 481: 1882, 491: 1882, 495: 1882, 1882, 1882, 534: 1882, 555: 1882, 1882, 1882, 1882, 560: 1882, 1882, 1882, 1882, 1882, 1882, 567: 1882, 1882, 1882, 1882, 572: 1882, 1882, 575: 1882, 577: 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 1882, 636: 1882}, + {2: 1881, 1881, 1881, 1881, 1881, 8: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 51: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 462: 1881, 464: 1881, 1881, 1881, 468: 1881, 471: 1881, 1881, 474: 1881, 1881, 1881, 481: 1881, 491: 1881, 495: 1881, 1881, 1881, 534: 1881, 555: 1881, 1881, 1881, 1881, 560: 1881, 1881, 1881, 1881, 1881, 1881, 567: 1881, 1881, 1881, 1881, 572: 1881, 1881, 575: 1881, 577: 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 1881, 636: 1881}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3248, 3154, 3235, 3153, 3150}, + {50: 3252, 473: 3250, 571: 3251}, // 790 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3249, 3153, 3236, 3152, 3149}, - {50: 3253, 473: 3251, 571: 3252}, - {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 497: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 533: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3388, 653: 3390, 2674, 2675, 2673, 728: 3387, 858: 3386}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3385, 3153, 3236, 3152, 3149}, + {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 721, 498: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 535: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3389, 651: 3391, 2675, 2676, 2674, 726: 3388, 856: 3387}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3386, 3154, 3235, 3153, 3150}, + {144: 907, 479: 907, 486: 3254, 728: 907, 1237: 3253}, + {144: 3258, 479: 3259, 728: 910, 869: 3257}, // 795 - {144: 907, 479: 907, 486: 3255, 730: 907, 1239: 3254}, - {144: 3259, 479: 3260, 730: 910, 871: 3258}, - {8: 3256, 338: 3257}, - {144: 906, 479: 906, 730: 906}, - {144: 905, 479: 905, 730: 905}, - // 800 - {730: 3263, 737: 3264}, - {260: 3262}, + {8: 3255, 340: 3256}, + {144: 906, 479: 906, 728: 906}, + {144: 905, 479: 905, 728: 905}, + {728: 3262, 735: 3263}, {260: 3261}, - {730: 908}, - {730: 909}, + // 800 + {260: 3260}, + {728: 908}, + {728: 909}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 3301, 651: 3300, 2675, 2676, 2674, 908: 3303, 1140: 3304, 1321: 3302}, + {916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 463: 916, 916, 916, 916, 916, 469: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 482: 916, 916, 916, 916, 916, 916, 916, 916, 916, 492: 916, 916, 916, 498: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 535: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 571: 916}, // 805 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 3300, 653: 3299, 2674, 2675, 2673, 910: 3302, 1141: 3303, 1323: 3301}, - {916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 463: 916, 916, 916, 916, 916, 469: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 482: 916, 916, 916, 916, 916, 916, 916, 916, 916, 492: 916, 916, 497: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 533: 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 916, 571: 916}, - {1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 657: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789}, - {1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 657: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783}, - {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 657: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773}, + {1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 655: 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789, 1789}, + {1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 655: 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783, 1783}, + {1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 655: 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773, 1773}, + {1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 655: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762}, + {1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 655: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751}, // 810 - {1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 657: 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762, 1762}, - {1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 657: 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751, 1751}, - {1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 657: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749}, - {1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 657: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726}, - {1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 657: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720}, + {1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 655: 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749, 1749}, + {1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 655: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726}, + {1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 655: 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720, 1720}, + {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 655: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710}, + {1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 655: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685}, // 815 - {1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 657: 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710, 1710}, - {1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 657: 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685, 1685}, - {1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 657: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684}, - {1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 657: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681}, - {1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 657: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676}, + {1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 655: 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684, 1684}, + {1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 655: 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681, 1681}, + {1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 655: 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676, 1676}, + {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 655: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674}, + {1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 655: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673}, // 820 - {1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 657: 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674, 1674}, - {1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 657: 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673, 1673}, - {1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 657: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670}, - {1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 657: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668}, - {1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 657: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655}, + {1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 655: 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670, 1670}, + {1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 655: 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668, 1668}, + {1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 655: 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655, 1655}, + {1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 655: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634}, + {1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 655: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617}, // 825 - {1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 657: 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634, 1634}, - {1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 657: 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617, 1617}, - {1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 657: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616}, - {1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 657: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615}, - {1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 657: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611}, + {1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 655: 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616, 1616}, + {1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 655: 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615, 1615}, + {1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 655: 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611, 1611}, + {1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 655: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610}, + {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 655: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605}, // 830 - {1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 657: 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610, 1610}, - {1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 657: 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605, 1605}, - {1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 657: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496}, - {1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 657: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495}, - {1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 657: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494}, + {1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 655: 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496, 1496}, + {1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 655: 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495, 1495}, + {1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 655: 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494, 1494}, + {1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 655: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493}, + {1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 655: 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412}, // 835 - {1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 657: 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493, 1493}, - {1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 657: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397}, - {1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 657: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367}, - {1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 657: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366}, - {1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 657: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347}, + {1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 655: 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397, 1397}, + {1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 655: 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379, 1379}, + {1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 655: 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367, 1367}, + {1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 655: 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366, 1366}, + {1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 655: 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347, 1347}, // 840 - {1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 657: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346}, - {958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 463: 958, 958, 958, 958, 958, 469: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 482: 958, 958, 958, 958, 958, 958, 958, 958, 958, 492: 958, 958, 497: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 533: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 571: 958}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 955, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 480: 955, 490: 955, 513: 955, 516: 955, 955, 653: 3299, 2674, 2675, 2673, 910: 3306, 1238: 3305, 1324: 3304}, - {929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 463: 929, 929, 929, 929, 929, 469: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 482: 929, 929, 929, 929, 929, 929, 929, 929, 929, 492: 929, 929, 497: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 533: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 571: 929}, - {928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 463: 928, 928, 928, 928, 928, 469: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 482: 928, 928, 928, 928, 928, 928, 928, 928, 928, 492: 928, 928, 497: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 533: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 571: 928}, + {1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 655: 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346, 1346}, + {958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 463: 958, 958, 958, 958, 958, 469: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 482: 958, 958, 958, 958, 958, 958, 958, 958, 958, 492: 958, 958, 958, 498: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 535: 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 958, 571: 958}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 955, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 480: 955, 490: 955, 513: 955, 516: 955, 955, 651: 3300, 2675, 2676, 2674, 908: 3307, 1236: 3306, 1322: 3305}, + {929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 463: 929, 929, 929, 929, 929, 469: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 482: 929, 929, 929, 929, 929, 929, 929, 929, 929, 492: 929, 929, 929, 498: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 535: 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 929, 571: 929}, + {928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 463: 928, 928, 928, 928, 928, 469: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 482: 928, 928, 928, 928, 928, 928, 928, 928, 928, 492: 928, 928, 928, 498: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 535: 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 928, 571: 928}, // 845 - {927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 463: 927, 927, 927, 927, 927, 469: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 482: 927, 927, 927, 927, 927, 927, 927, 927, 927, 492: 927, 927, 497: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 533: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 571: 927}, - {50: 3384}, - {50: 953, 480: 3308, 490: 953, 513: 953, 516: 953, 953, 1241: 3307}, + {927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 463: 927, 927, 927, 927, 927, 469: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 482: 927, 927, 927, 927, 927, 927, 927, 927, 927, 492: 927, 927, 927, 498: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 535: 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 927, 571: 927}, + {50: 3385}, + {50: 953, 480: 3309, 490: 953, 513: 953, 516: 953, 953, 1239: 3308}, {50: 954, 480: 954, 490: 954, 513: 954, 516: 954, 954}, - {50: 951, 490: 3314, 513: 951, 516: 951, 951, 1244: 3313}, + {50: 951, 490: 3315, 513: 951, 516: 951, 951, 1242: 3314}, // 850 - {649: 3309}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 2666, 857: 3142, 887: 3310}, - {7: 3311, 50: 952, 490: 952, 513: 952, 516: 952, 952}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 2666, 857: 3312}, + {647: 3310}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 2667, 855: 3143, 885: 3311}, + {7: 3312, 50: 952, 490: 952, 513: 952, 516: 952, 952}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 2667, 855: 3313}, {1273, 1273, 7: 1273, 50: 1273, 124: 1273, 461: 1273, 463: 1273, 469: 1273, 1273, 477: 1273, 1273, 482: 1273, 1273, 1273, 1273, 488: 1273, 490: 1273, 504: 1273, 1273, 513: 1273, 516: 1273, 1273}, // 855 - {50: 949, 513: 3319, 516: 3320, 3321, 1243: 3317, 1322: 3318}, - {649: 3315}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 2666, 857: 3142, 887: 3316}, - {7: 3311, 50: 950, 513: 950, 516: 950, 950}, + {50: 949, 513: 3320, 516: 3321, 3322, 1241: 3318, 1320: 3319}, + {647: 3316}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 2667, 855: 3143, 885: 3317}, + {7: 3312, 50: 950, 513: 950, 516: 950, 950}, {50: 956}, // 860 - {146: 3332, 161: 3328, 496: 3322, 544: 3333, 562: 3324, 3323, 567: 3330, 570: 3331, 807: 3329, 963: 3326, 1320: 3327, 3325}, - {146: 947, 161: 947, 496: 947, 544: 947, 562: 947, 947, 567: 947, 570: 947}, - {146: 946, 161: 946, 496: 946, 544: 946, 562: 946, 946, 567: 946, 570: 946}, - {146: 945, 161: 945, 496: 945, 544: 945, 562: 945, 945, 567: 945, 570: 945}, - {2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 50: 2161, 130: 2161, 148: 2161, 461: 2161, 2161, 2161, 465: 2161, 2161, 2161, 2161, 2161, 473: 2161, 479: 2161, 2161, 2161, 491: 2161, 2161, 494: 2161, 2161, 559: 2161, 566: 2161, 574: 2161, 576: 2161, 632: 2161, 2161, 2161, 2161, 2161, 2161, 639: 2161}, + {146: 3333, 161: 3329, 497: 3323, 544: 3334, 562: 3325, 3324, 567: 3331, 570: 3332, 805: 3330, 961: 3327, 1318: 3328, 3326}, + {146: 947, 161: 947, 497: 947, 544: 947, 562: 947, 947, 567: 947, 570: 947}, + {146: 946, 161: 946, 497: 946, 544: 946, 562: 946, 946, 567: 946, 570: 946}, + {146: 945, 161: 945, 497: 945, 544: 945, 562: 945, 945, 567: 945, 570: 945}, + {2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 50: 2162, 130: 2162, 148: 2162, 461: 2162, 2162, 2162, 465: 2162, 2162, 2162, 2162, 2162, 473: 2162, 479: 2162, 2162, 2162, 491: 2162, 2162, 495: 2162, 2162, 559: 2162, 566: 2162, 574: 2162, 576: 2162, 630: 2162, 2162, 2162, 2162, 2162, 2162, 637: 2162}, // 865 - {2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 50: 2160, 130: 2160, 148: 2160, 185: 2160, 461: 2160, 2160, 2160, 465: 2160, 2160, 2160, 2160, 2160, 473: 2160, 479: 2160, 2160, 2160, 491: 2160, 2160, 494: 2160, 2160, 559: 2160, 566: 2160, 574: 2160, 576: 2160, 632: 2160, 2160, 2160, 2160, 2160, 2160, 639: 2160}, - {2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 2159, 50: 2159, 130: 2159, 148: 2159, 185: 2159, 461: 2159, 2159, 2159, 465: 2159, 2159, 2159, 2159, 2159, 473: 2159, 479: 2159, 2159, 2159, 491: 2159, 2159, 494: 2159, 2159, 559: 2159, 566: 2159, 574: 2159, 576: 2159, 632: 2159, 2159, 2159, 2159, 2159, 2159, 639: 2159}, + {2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 2161, 50: 2161, 130: 2161, 148: 2161, 186: 2161, 461: 2161, 2161, 2161, 465: 2161, 2161, 2161, 2161, 2161, 473: 2161, 479: 2161, 2161, 2161, 491: 2161, 2161, 495: 2161, 2161, 559: 2161, 566: 2161, 574: 2161, 576: 2161, 630: 2161, 2161, 2161, 2161, 2161, 2161, 637: 2161}, + {2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 2160, 50: 2160, 130: 2160, 148: 2160, 186: 2160, 461: 2160, 2160, 2160, 465: 2160, 2160, 2160, 2160, 2160, 473: 2160, 479: 2160, 2160, 2160, 491: 2160, 2160, 495: 2160, 2160, 559: 2160, 566: 2160, 574: 2160, 576: 2160, 630: 2160, 2160, 2160, 2160, 2160, 2160, 637: 2160}, {50: 948}, {50: 944}, {50: 943}, // 870 - {130: 3379}, - {130: 3377}, - {130: 3375}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3382}, - {564: 3381}, + {130: 3380}, + {130: 3378}, + {130: 3376}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3383}, + {564: 3382}, // 875 - {146: 3332, 161: 3334, 496: 3322, 562: 3324, 3323, 567: 3336, 570: 3337, 807: 3335, 963: 3339, 1140: 3338}, - {130: 3379, 148: 3380}, - {130: 3377, 148: 3378}, - {130: 3375, 148: 3376}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3342}, + {146: 3333, 161: 3335, 497: 3323, 562: 3325, 3324, 567: 3337, 570: 3338, 805: 3336, 961: 3340, 1139: 3339}, + {130: 3380, 148: 3381}, + {130: 3378, 148: 3379}, + {130: 3376, 148: 3377}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3343}, // 880 - {493: 3340}, - {50: 936, 493: 936}, - {146: 3332, 161: 3334, 496: 3322, 562: 3324, 3323, 567: 3336, 570: 3337, 807: 3335, 963: 3339, 1140: 3341}, + {494: 3341}, + {50: 936, 494: 936}, + {146: 3333, 161: 3335, 497: 3323, 562: 3325, 3324, 567: 3337, 570: 3338, 805: 3336, 961: 3340, 1139: 3342}, {50: 937}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3372}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3373}, // 885 - {1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 469: 1079, 1079, 1079, 1079, 474: 1079, 1079, 1079, 1079, 1079, 1079, 482: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 495: 1079, 497: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 533: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 559: 1079, 636: 1079}, - {1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 469: 1078, 1078, 1078, 1078, 474: 1078, 1078, 1078, 1078, 1078, 1078, 482: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 495: 1078, 497: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 533: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 559: 1078, 636: 1078}, - {1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 469: 1077, 1077, 1077, 1077, 474: 1077, 1077, 1077, 1077, 1077, 1077, 482: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 495: 1077, 497: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 533: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 559: 1077, 636: 1077}, - {1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 469: 1076, 1076, 1076, 1076, 474: 1076, 1076, 1076, 1076, 1076, 1076, 482: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 495: 1076, 497: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 533: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 559: 1076, 636: 1076}, - {1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 469: 1075, 1075, 1075, 1075, 474: 1075, 1075, 1075, 1075, 1075, 1075, 482: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 495: 1075, 497: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 533: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 559: 1075, 636: 1075}, + {1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 469: 1079, 1079, 1079, 1079, 474: 1079, 1079, 1079, 1079, 1079, 1079, 482: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 496: 1079, 498: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 535: 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 1079, 559: 1079, 634: 1079}, + {1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 469: 1078, 1078, 1078, 1078, 474: 1078, 1078, 1078, 1078, 1078, 1078, 482: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 496: 1078, 498: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 535: 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 1078, 559: 1078, 634: 1078}, + {1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 469: 1077, 1077, 1077, 1077, 474: 1077, 1077, 1077, 1077, 1077, 1077, 482: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 496: 1077, 498: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 535: 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 1077, 559: 1077, 634: 1077}, + {1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 469: 1076, 1076, 1076, 1076, 474: 1076, 1076, 1076, 1076, 1076, 1076, 482: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 496: 1076, 498: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 535: 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 1076, 559: 1076, 634: 1076}, + {1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 469: 1075, 1075, 1075, 1075, 474: 1075, 1075, 1075, 1075, 1075, 1075, 482: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 496: 1075, 498: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 535: 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 1075, 559: 1075, 634: 1075}, // 890 - {1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 469: 1074, 1074, 1074, 1074, 474: 1074, 1074, 1074, 1074, 1074, 1074, 482: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 495: 1074, 497: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 533: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 559: 1074, 636: 1074}, - {1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 469: 1073, 1073, 1073, 1073, 474: 1073, 1073, 1073, 1073, 1073, 1073, 482: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 495: 1073, 497: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 533: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 559: 1073, 636: 1073}, - {1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 469: 1072, 1072, 1072, 1072, 474: 1072, 1072, 1072, 1072, 1072, 1072, 482: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 495: 1072, 497: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 533: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 559: 1072, 636: 1072}, - {1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 469: 1071, 1071, 1071, 1071, 474: 1071, 1071, 1071, 1071, 1071, 1071, 482: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 495: 1071, 497: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 533: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 559: 1071, 636: 1071}, - {1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 469: 1070, 1070, 1070, 1070, 474: 1070, 1070, 1070, 1070, 1070, 1070, 482: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 495: 1070, 497: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 533: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 559: 1070, 636: 1070}, + {1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 469: 1074, 1074, 1074, 1074, 474: 1074, 1074, 1074, 1074, 1074, 1074, 482: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 496: 1074, 498: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 535: 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 1074, 559: 1074, 634: 1074}, + {1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 469: 1073, 1073, 1073, 1073, 474: 1073, 1073, 1073, 1073, 1073, 1073, 482: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 496: 1073, 498: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 535: 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 1073, 559: 1073, 634: 1073}, + {1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 469: 1072, 1072, 1072, 1072, 474: 1072, 1072, 1072, 1072, 1072, 1072, 482: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 496: 1072, 498: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 535: 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 1072, 559: 1072, 634: 1072}, + {1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 469: 1071, 1071, 1071, 1071, 474: 1071, 1071, 1071, 1071, 1071, 1071, 482: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 496: 1071, 498: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 535: 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 1071, 559: 1071, 634: 1071}, + {1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 469: 1070, 1070, 1070, 1070, 474: 1070, 1070, 1070, 1070, 1070, 1070, 482: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 496: 1070, 498: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 535: 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 1070, 559: 1070, 634: 1070}, // 895 - {1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 469: 1069, 1069, 1069, 1069, 474: 1069, 1069, 1069, 1069, 1069, 1069, 482: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 495: 1069, 497: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 533: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 559: 1069, 636: 1069}, - {1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 469: 1068, 1068, 1068, 1068, 474: 1068, 1068, 1068, 1068, 1068, 1068, 482: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 495: 1068, 497: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 533: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 559: 1068, 636: 1068}, - {1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 469: 1067, 1067, 1067, 1067, 474: 1067, 1067, 1067, 1067, 1067, 1067, 482: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 495: 1067, 497: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 533: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 559: 1067, 636: 1067}, - {1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 469: 1066, 1066, 1066, 1066, 474: 1066, 1066, 1066, 1066, 1066, 1066, 482: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 495: 1066, 497: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 533: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 559: 1066, 636: 1066}, - {1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 469: 1065, 1065, 1065, 1065, 474: 1065, 1065, 1065, 1065, 1065, 1065, 482: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 495: 1065, 497: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 533: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 559: 1065, 636: 1065}, + {1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 469: 1069, 1069, 1069, 1069, 474: 1069, 1069, 1069, 1069, 1069, 1069, 482: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 496: 1069, 498: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 535: 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 1069, 559: 1069, 634: 1069}, + {1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 469: 1068, 1068, 1068, 1068, 474: 1068, 1068, 1068, 1068, 1068, 1068, 482: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 496: 1068, 498: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 535: 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 1068, 559: 1068, 634: 1068}, + {1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 469: 1067, 1067, 1067, 1067, 474: 1067, 1067, 1067, 1067, 1067, 1067, 482: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 496: 1067, 498: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 535: 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 1067, 559: 1067, 634: 1067}, + {1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 469: 1066, 1066, 1066, 1066, 474: 1066, 1066, 1066, 1066, 1066, 1066, 482: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 496: 1066, 498: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 535: 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 1066, 559: 1066, 634: 1066}, + {1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 469: 1065, 1065, 1065, 1065, 474: 1065, 1065, 1065, 1065, 1065, 1065, 482: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 496: 1065, 498: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 535: 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 1065, 559: 1065, 634: 1065}, // 900 - {1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 469: 1064, 1064, 1064, 1064, 474: 1064, 1064, 1064, 1064, 1064, 1064, 482: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 495: 1064, 497: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 533: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 559: 1064, 636: 1064}, - {1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 469: 1063, 1063, 1063, 1063, 474: 1063, 1063, 1063, 1063, 1063, 1063, 482: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 495: 1063, 497: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 533: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 559: 1063, 636: 1063}, - {1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 469: 1062, 1062, 1062, 1062, 474: 1062, 1062, 1062, 1062, 1062, 1062, 482: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 495: 1062, 497: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 533: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 559: 1062, 636: 1062}, - {1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 469: 1061, 1061, 1061, 1061, 474: 1061, 1061, 1061, 1061, 1061, 1061, 482: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 495: 1061, 497: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 533: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 559: 1061, 636: 1061}, - {1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 469: 1060, 1060, 1060, 1060, 474: 1060, 1060, 1060, 1060, 1060, 1060, 482: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 495: 1060, 497: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 533: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 559: 1060, 636: 1060}, + {1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 469: 1064, 1064, 1064, 1064, 474: 1064, 1064, 1064, 1064, 1064, 1064, 482: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 496: 1064, 498: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 535: 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 1064, 559: 1064, 634: 1064}, + {1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 469: 1063, 1063, 1063, 1063, 474: 1063, 1063, 1063, 1063, 1063, 1063, 482: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 496: 1063, 498: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 535: 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 1063, 559: 1063, 634: 1063}, + {1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 469: 1062, 1062, 1062, 1062, 474: 1062, 1062, 1062, 1062, 1062, 1062, 482: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 496: 1062, 498: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 535: 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 1062, 559: 1062, 634: 1062}, + {1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 469: 1061, 1061, 1061, 1061, 474: 1061, 1061, 1061, 1061, 1061, 1061, 482: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 496: 1061, 498: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 535: 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 1061, 559: 1061, 634: 1061}, + {1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 469: 1060, 1060, 1060, 1060, 474: 1060, 1060, 1060, 1060, 1060, 1060, 482: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 496: 1060, 498: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 535: 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 1060, 559: 1060, 634: 1060}, // 905 - {1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 469: 1059, 1059, 1059, 1059, 474: 1059, 1059, 1059, 1059, 1059, 1059, 482: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 495: 1059, 497: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 533: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 559: 1059, 636: 1059}, - {1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 469: 1058, 1058, 1058, 1058, 474: 1058, 1058, 1058, 1058, 1058, 1058, 482: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 495: 1058, 497: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 533: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 559: 1058, 636: 1058}, - {1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 469: 1057, 1057, 1057, 1057, 474: 1057, 1057, 1057, 1057, 1057, 1057, 482: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 495: 1057, 497: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 533: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 559: 1057, 636: 1057}, - {1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 469: 1056, 1056, 1056, 1056, 474: 1056, 1056, 1056, 1056, 1056, 1056, 482: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 495: 1056, 497: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 533: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 559: 1056, 636: 1056}, - {1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 469: 1055, 1055, 1055, 1055, 474: 1055, 1055, 1055, 1055, 1055, 1055, 482: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 495: 1055, 497: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 533: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 559: 1055, 636: 1055}, + {1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 469: 1059, 1059, 1059, 1059, 474: 1059, 1059, 1059, 1059, 1059, 1059, 482: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 496: 1059, 498: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 535: 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 1059, 559: 1059, 634: 1059}, + {1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 469: 1058, 1058, 1058, 1058, 474: 1058, 1058, 1058, 1058, 1058, 1058, 482: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 496: 1058, 498: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 535: 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 1058, 559: 1058, 634: 1058}, + {1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 469: 1057, 1057, 1057, 1057, 474: 1057, 1057, 1057, 1057, 1057, 1057, 482: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 496: 1057, 498: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 535: 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 1057, 559: 1057, 634: 1057}, + {1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 469: 1056, 1056, 1056, 1056, 474: 1056, 1056, 1056, 1056, 1056, 1056, 482: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 496: 1056, 498: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 535: 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 1056, 559: 1056, 634: 1056}, + {1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 469: 1055, 1055, 1055, 1055, 474: 1055, 1055, 1055, 1055, 1055, 1055, 482: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 496: 1055, 498: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 535: 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 1055, 559: 1055, 634: 1055}, // 910 - {1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 469: 1054, 1054, 1054, 1054, 474: 1054, 1054, 1054, 1054, 1054, 1054, 482: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 495: 1054, 497: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 533: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 559: 1054, 636: 1054}, - {1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 469: 1053, 1053, 1053, 1053, 474: 1053, 1053, 1053, 1053, 1053, 1053, 482: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 495: 1053, 497: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 533: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 559: 1053, 636: 1053}, - {1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 469: 1052, 1052, 1052, 1052, 474: 1052, 1052, 1052, 1052, 1052, 1052, 482: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 495: 1052, 497: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 533: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 559: 1052, 636: 1052}, - {1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 469: 1051, 1051, 1051, 1051, 474: 1051, 1051, 1051, 1051, 1051, 1051, 482: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 495: 1051, 497: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 533: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 559: 1051, 636: 1051}, - {130: 3373, 148: 3374}, + {1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 469: 1054, 1054, 1054, 1054, 474: 1054, 1054, 1054, 1054, 1054, 1054, 482: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 496: 1054, 498: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 535: 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 1054, 559: 1054, 634: 1054}, + {1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 469: 1053, 1053, 1053, 1053, 474: 1053, 1053, 1053, 1053, 1053, 1053, 482: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 496: 1053, 498: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 535: 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 1053, 559: 1053, 634: 1053}, + {1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 469: 1052, 1052, 1052, 1052, 474: 1052, 1052, 1052, 1052, 1052, 1052, 482: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 496: 1052, 498: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 535: 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 1052, 559: 1052, 634: 1052}, + {1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 469: 1051, 1051, 1051, 1051, 474: 1051, 1051, 1051, 1051, 1051, 1051, 482: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 496: 1051, 498: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 535: 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 1051, 559: 1051, 634: 1051}, + {130: 3374, 148: 3375}, // 915 - {50: 939, 493: 939}, - {50: 932, 493: 932}, - {50: 940, 493: 940}, - {50: 933, 493: 933}, - {50: 941, 493: 941}, + {50: 939, 494: 939}, + {50: 932, 494: 932}, + {50: 940, 494: 940}, + {50: 933, 494: 933}, + {50: 941, 494: 941}, // 920 - {50: 934, 493: 934}, - {50: 942, 493: 942}, - {50: 935, 493: 935}, - {50: 938, 493: 938}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3383}, + {50: 934, 494: 934}, + {50: 942, 494: 942}, + {50: 935, 494: 935}, + {50: 938, 494: 938}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3384}, // 925 - {130: 3373}, - {957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 463: 957, 957, 957, 957, 957, 469: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 482: 957, 957, 957, 957, 957, 957, 957, 957, 957, 492: 957, 957, 497: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 533: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 571: 957}, - {1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 463: 1231, 1231, 1231, 1231, 1231, 469: 1231, 1231, 1231, 1231, 3251, 1231, 1231, 1231, 1231, 1231, 1231, 482: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 492: 1231, 1231, 497: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 533: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 571: 1231}, - {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 463: 1241, 1241, 1241, 1241, 1241, 469: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 482: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 492: 1241, 1241, 497: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 533: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 571: 1241}, - {728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 497: 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 533: 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 559: 728, 566: 728, 571: 728, 574: 728, 576: 728, 632: 728, 728, 728, 728, 728, 728, 639: 728}, + {130: 3374}, + {957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 463: 957, 957, 957, 957, 957, 469: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 482: 957, 957, 957, 957, 957, 957, 957, 957, 957, 492: 957, 957, 957, 498: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 535: 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 957, 571: 957}, + {1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 463: 1231, 1231, 1231, 1231, 1231, 469: 1231, 1231, 1231, 1231, 3250, 1231, 1231, 1231, 1231, 1231, 1231, 482: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 492: 1231, 1231, 1231, 498: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 535: 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 1231, 571: 1231}, + {1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 463: 1241, 1241, 1241, 1241, 1241, 469: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 482: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 492: 1241, 1241, 1241, 498: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 535: 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 1241, 571: 1241}, + {728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 498: 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 535: 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 728, 559: 728, 566: 728, 571: 728, 574: 728, 576: 728, 630: 728, 728, 728, 728, 728, 728, 637: 728}, // 930 - {727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 497: 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 533: 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 559: 727, 566: 727, 571: 727, 574: 727, 576: 727, 632: 727, 727, 727, 727, 727, 727, 639: 727}, - {248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 497: 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 557: 248, 559: 248, 566: 248, 571: 248, 574: 248, 576: 248, 632: 248, 248, 248, 248, 248, 248, 639: 248, 248, 643: 248, 648: 248, 248, 651: 248, 657: 248, 248}, - {247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 497: 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 557: 247, 559: 247, 566: 247, 571: 247, 574: 247, 576: 247, 632: 247, 247, 247, 247, 247, 247, 639: 247, 247, 643: 247, 648: 247, 247, 651: 247, 657: 247, 247}, - {1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 463: 1898, 1898, 467: 1898, 469: 1898, 1898, 1898, 1898, 477: 1898, 1898, 1898, 482: 1898, 1898, 1898, 1898, 1898, 488: 1898, 1898, 1898, 492: 1898, 1898, 497: 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 511: 1898, 513: 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 534: 1898, 1898, 723: 3243, 3241}, - {1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 463: 1899, 1899, 467: 1899, 469: 1899, 1899, 1899, 1899, 477: 1899, 1899, 1899, 482: 1899, 1899, 1899, 1899, 1899, 488: 1899, 1899, 1899, 492: 1899, 3247, 497: 1899, 3246, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 511: 1899, 513: 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 534: 1899, 1899, 723: 3243, 3241}, + {727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 498: 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 535: 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 727, 559: 727, 566: 727, 571: 727, 574: 727, 576: 727, 630: 727, 727, 727, 727, 727, 727, 637: 727}, + {248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 498: 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 248, 557: 248, 559: 248, 566: 248, 571: 248, 574: 248, 576: 248, 630: 248, 248, 248, 248, 248, 248, 637: 248, 248, 641: 248, 646: 248, 248, 649: 248, 655: 248, 248}, + {247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 498: 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 247, 557: 247, 559: 247, 566: 247, 571: 247, 574: 247, 576: 247, 630: 247, 247, 247, 247, 247, 247, 637: 247, 247, 641: 247, 646: 247, 247, 649: 247, 655: 247, 247}, + {1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 463: 1898, 1898, 467: 1898, 469: 1898, 1898, 1898, 1898, 477: 1898, 1898, 1898, 482: 1898, 1898, 1898, 1898, 1898, 488: 1898, 1898, 1898, 492: 1898, 1898, 1898, 498: 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 511: 1898, 513: 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 1898, 533: 1898, 535: 1898, 722: 3242, 3240}, + {1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 463: 1899, 1899, 467: 1899, 469: 1899, 1899, 1899, 1899, 477: 1899, 1899, 1899, 482: 1899, 1899, 1899, 1899, 1899, 488: 1899, 1899, 1899, 492: 1899, 1899, 3246, 498: 1899, 3245, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 511: 1899, 513: 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 1899, 533: 1899, 535: 1899, 722: 3242, 3240}, // 935 - {1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 463: 1900, 1900, 467: 1900, 469: 1900, 1900, 1900, 1900, 477: 1900, 1900, 1900, 482: 1900, 1900, 1900, 1900, 1900, 488: 1900, 1900, 1900, 492: 1900, 3247, 497: 1900, 3246, 1900, 3242, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 511: 1900, 513: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 534: 1900, 1900, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3395}, - {50: 3396, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {144: 3259, 479: 3260, 730: 910, 871: 3397}, - {730: 3263, 737: 3398}, + {1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 463: 1900, 1900, 467: 1900, 469: 1900, 1900, 1900, 1900, 477: 1900, 1900, 1900, 482: 1900, 1900, 1900, 1900, 1900, 488: 1900, 1900, 1900, 492: 1900, 1900, 3246, 498: 1900, 3245, 1900, 3241, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 511: 1900, 513: 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 1900, 533: 1900, 535: 1900, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3396}, + {50: 3397, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {144: 3258, 479: 3259, 728: 910, 869: 3398}, + {728: 3262, 735: 3399}, // 940 - {917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 463: 917, 917, 917, 917, 917, 469: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 482: 917, 917, 917, 917, 917, 917, 917, 917, 917, 492: 917, 917, 497: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 533: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 571: 917}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3400}, - {50: 3401, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {144: 3259, 479: 3260, 730: 910, 871: 3402}, - {730: 3263, 737: 3403}, + {917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 463: 917, 917, 917, 917, 917, 469: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 482: 917, 917, 917, 917, 917, 917, 917, 917, 917, 492: 917, 917, 917, 498: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 535: 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 917, 571: 917}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3401}, + {50: 3402, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {144: 3258, 479: 3259, 728: 910, 869: 3403}, + {728: 3262, 735: 3404}, // 945 - {918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 463: 918, 918, 918, 918, 918, 469: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 482: 918, 918, 918, 918, 918, 918, 918, 918, 918, 492: 918, 918, 497: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 533: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 571: 918}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3405}, - {7: 3407, 50: 915, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241, 1075: 3406}, - {50: 3414}, - {496: 3322, 562: 3324, 3323, 567: 3409, 807: 3408}, + {918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 463: 918, 918, 918, 918, 918, 469: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 482: 918, 918, 918, 918, 918, 918, 918, 918, 918, 492: 918, 918, 918, 498: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 535: 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, 571: 918}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3406}, + {7: 3408, 50: 915, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240, 1074: 3407}, + {50: 3415}, + {497: 3323, 562: 3325, 3324, 567: 3410, 805: 3409}, // 950 - {7: 3411, 50: 912, 1076: 3413}, - {7: 3411, 50: 912, 1076: 3410}, + {7: 3412, 50: 912, 1075: 3414}, + {7: 3412, 50: 912, 1075: 3411}, {50: 913}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3412}, - {50: 911, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3413}, + {50: 911, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 955 {50: 914}, - {144: 3259, 479: 3260, 730: 910, 871: 3415}, - {730: 3263, 737: 3416}, - {919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 463: 919, 919, 919, 919, 919, 469: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 482: 919, 919, 919, 919, 919, 919, 919, 919, 919, 492: 919, 919, 497: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 533: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 571: 919}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3418}, + {144: 3258, 479: 3259, 728: 910, 869: 3416}, + {728: 3262, 735: 3417}, + {919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 463: 919, 919, 919, 919, 919, 469: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 482: 919, 919, 919, 919, 919, 919, 919, 919, 919, 492: 919, 919, 919, 498: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 535: 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 919, 571: 919}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3419}, // 960 - {7: 3407, 50: 915, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241, 1075: 3419}, - {50: 3420}, - {144: 3259, 479: 3260, 730: 910, 871: 3421}, - {730: 3263, 737: 3422}, - {920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 463: 920, 920, 920, 920, 920, 469: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 482: 920, 920, 920, 920, 920, 920, 920, 920, 920, 492: 920, 920, 497: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 533: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 571: 920}, + {7: 3408, 50: 915, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240, 1074: 3420}, + {50: 3421}, + {144: 3258, 479: 3259, 728: 910, 869: 3422}, + {728: 3262, 735: 3423}, + {920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 463: 920, 920, 920, 920, 920, 469: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 482: 920, 920, 920, 920, 920, 920, 920, 920, 920, 492: 920, 920, 920, 498: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 535: 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 920, 571: 920}, // 965 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3424, 3153, 3236, 3152, 3149}, - {50: 3425, 473: 3251, 571: 3252}, - {730: 3263, 737: 3426}, - {921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 463: 921, 921, 921, 921, 921, 469: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 482: 921, 921, 921, 921, 921, 921, 921, 921, 921, 492: 921, 921, 497: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 533: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 571: 921}, - {50: 3428}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3425, 3154, 3235, 3153, 3150}, + {50: 3426, 473: 3250, 571: 3251}, + {728: 3262, 735: 3427}, + {921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 463: 921, 921, 921, 921, 921, 469: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 482: 921, 921, 921, 921, 921, 921, 921, 921, 921, 492: 921, 921, 921, 498: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 535: 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 921, 571: 921}, + {50: 3429}, // 970 - {730: 3263, 737: 3429}, - {922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 463: 922, 922, 922, 922, 922, 469: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 482: 922, 922, 922, 922, 922, 922, 922, 922, 922, 492: 922, 922, 497: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 533: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 571: 922}, - {50: 3431}, - {730: 3263, 737: 3432}, - {923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 463: 923, 923, 923, 923, 923, 469: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 482: 923, 923, 923, 923, 923, 923, 923, 923, 923, 492: 923, 923, 497: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 533: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 571: 923}, + {728: 3262, 735: 3430}, + {922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 463: 922, 922, 922, 922, 922, 469: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 482: 922, 922, 922, 922, 922, 922, 922, 922, 922, 492: 922, 922, 922, 498: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 535: 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 922, 571: 922}, + {50: 3432}, + {728: 3262, 735: 3433}, + {923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 463: 923, 923, 923, 923, 923, 469: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 482: 923, 923, 923, 923, 923, 923, 923, 923, 923, 492: 923, 923, 923, 498: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 535: 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 923, 571: 923}, // 975 - {50: 3434}, - {730: 3263, 737: 3435}, - {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 463: 924, 924, 924, 924, 924, 469: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 482: 924, 924, 924, 924, 924, 924, 924, 924, 924, 492: 924, 924, 497: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 533: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 571: 924}, - {50: 3437}, - {730: 3263, 737: 3438}, + {50: 3435}, + {728: 3262, 735: 3436}, + {924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 463: 924, 924, 924, 924, 924, 469: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 482: 924, 924, 924, 924, 924, 924, 924, 924, 924, 492: 924, 924, 924, 498: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 535: 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 924, 571: 924}, + {50: 3438}, + {728: 3262, 735: 3439}, // 980 - {925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 463: 925, 925, 925, 925, 925, 469: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 482: 925, 925, 925, 925, 925, 925, 925, 925, 925, 492: 925, 925, 497: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 533: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 571: 925}, - {50: 3440}, - {730: 3263, 737: 3441}, - {926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 463: 926, 926, 926, 926, 926, 469: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 482: 926, 926, 926, 926, 926, 926, 926, 926, 926, 492: 926, 926, 497: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 533: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 571: 926}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3449, 804: 3448}, + {925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 463: 925, 925, 925, 925, 925, 469: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 482: 925, 925, 925, 925, 925, 925, 925, 925, 925, 492: 925, 925, 925, 498: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 535: 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 925, 571: 925}, + {50: 3441}, + {728: 3262, 735: 3442}, + {926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 463: 926, 926, 926, 926, 926, 469: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 482: 926, 926, 926, 926, 926, 926, 926, 926, 926, 492: 926, 926, 926, 498: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 535: 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 926, 571: 926}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3450, 802: 3449}, // 985 - {2: 1214, 1214, 1214, 1214, 1214, 8: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 51: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 462: 1214, 464: 1214, 1214, 1214, 468: 1214, 471: 1214, 1214, 474: 1214, 1214, 1214, 481: 1214, 491: 1214, 494: 1214, 1214, 1214, 503: 1214, 512: 1214, 532: 1214, 555: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 567: 1214, 1214, 1214, 1214, 572: 1214, 1214, 575: 1214, 577: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 636: 1214, 638: 1214, 641: 1214, 735: 1214, 1214, 738: 1214, 1214, 1214, 749: 1214, 758: 1214, 1214, 1214}, - {2: 1213, 1213, 1213, 1213, 1213, 8: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 51: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 462: 1213, 464: 1213, 1213, 1213, 468: 1213, 471: 1213, 1213, 474: 1213, 1213, 1213, 481: 1213, 491: 1213, 494: 1213, 1213, 1213, 503: 1213, 512: 1213, 532: 1213, 555: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 567: 1213, 1213, 1213, 1213, 572: 1213, 1213, 575: 1213, 577: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 636: 1213, 638: 1213, 641: 1213, 735: 1213, 1213, 738: 1213, 1213, 1213, 749: 1213, 758: 1213, 1213, 1213}, - {2: 1212, 1212, 1212, 1212, 1212, 8: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 51: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 462: 1212, 464: 1212, 1212, 1212, 468: 1212, 471: 1212, 1212, 474: 1212, 1212, 1212, 481: 1212, 491: 1212, 494: 1212, 1212, 1212, 503: 1212, 512: 1212, 532: 1212, 555: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 567: 1212, 1212, 1212, 1212, 572: 1212, 1212, 575: 1212, 577: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 636: 1212, 638: 1212, 641: 1212, 735: 1212, 1212, 738: 1212, 1212, 1212, 749: 1212, 758: 1212, 1212, 1212}, - {2: 1211, 1211, 1211, 1211, 1211, 8: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 51: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 462: 1211, 464: 1211, 1211, 1211, 468: 1211, 471: 1211, 1211, 474: 1211, 1211, 1211, 481: 1211, 491: 1211, 494: 1211, 1211, 1211, 532: 1211, 555: 1211, 1211, 1211, 1211, 560: 1211, 1211, 1211, 1211, 1211, 1211, 567: 1211, 1211, 1211, 1211, 572: 1211, 1211, 575: 1211, 577: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 638: 1211, 641: 3454}, - {2: 1209, 1209, 1209, 1209, 1209, 8: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 51: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 462: 1209, 464: 1209, 1209, 1209, 468: 1209, 471: 1209, 1209, 474: 1209, 1209, 1209, 481: 1209, 491: 1209, 494: 1209, 1209, 1209, 532: 1209, 555: 1209, 1209, 1209, 1209, 560: 1209, 1209, 1209, 1209, 1209, 1209, 567: 1209, 1209, 1209, 1209, 572: 1209, 1209, 575: 1209, 577: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 638: 1209}, + {2: 1214, 1214, 1214, 1214, 1214, 8: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 51: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 462: 1214, 464: 1214, 1214, 1214, 468: 1214, 471: 1214, 1214, 474: 1214, 1214, 1214, 481: 1214, 491: 1214, 495: 1214, 1214, 1214, 503: 1214, 512: 1214, 534: 1214, 555: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 567: 1214, 1214, 1214, 1214, 572: 1214, 1214, 575: 1214, 577: 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 1214, 634: 1214, 636: 1214, 639: 1214, 733: 1214, 1214, 736: 1214, 1214, 1214, 747: 1214, 756: 1214, 1214, 1214}, + {2: 1213, 1213, 1213, 1213, 1213, 8: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 51: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 462: 1213, 464: 1213, 1213, 1213, 468: 1213, 471: 1213, 1213, 474: 1213, 1213, 1213, 481: 1213, 491: 1213, 495: 1213, 1213, 1213, 503: 1213, 512: 1213, 534: 1213, 555: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 567: 1213, 1213, 1213, 1213, 572: 1213, 1213, 575: 1213, 577: 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 1213, 634: 1213, 636: 1213, 639: 1213, 733: 1213, 1213, 736: 1213, 1213, 1213, 747: 1213, 756: 1213, 1213, 1213}, + {2: 1212, 1212, 1212, 1212, 1212, 8: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 51: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 462: 1212, 464: 1212, 1212, 1212, 468: 1212, 471: 1212, 1212, 474: 1212, 1212, 1212, 481: 1212, 491: 1212, 495: 1212, 1212, 1212, 503: 1212, 512: 1212, 534: 1212, 555: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 567: 1212, 1212, 1212, 1212, 572: 1212, 1212, 575: 1212, 577: 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 1212, 634: 1212, 636: 1212, 639: 1212, 733: 1212, 1212, 736: 1212, 1212, 1212, 747: 1212, 756: 1212, 1212, 1212}, + {2: 1211, 1211, 1211, 1211, 1211, 8: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 51: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 462: 1211, 464: 1211, 1211, 1211, 468: 1211, 471: 1211, 1211, 474: 1211, 1211, 1211, 481: 1211, 491: 1211, 495: 1211, 1211, 1211, 534: 1211, 555: 1211, 1211, 1211, 1211, 560: 1211, 1211, 1211, 1211, 1211, 1211, 567: 1211, 1211, 1211, 1211, 572: 1211, 1211, 575: 1211, 577: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 636: 1211, 639: 3455}, + {2: 1209, 1209, 1209, 1209, 1209, 8: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 51: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 462: 1209, 464: 1209, 1209, 1209, 468: 1209, 471: 1209, 1209, 474: 1209, 1209, 1209, 481: 1209, 491: 1209, 495: 1209, 1209, 1209, 534: 1209, 555: 1209, 1209, 1209, 1209, 560: 1209, 1209, 1209, 1209, 1209, 1209, 567: 1209, 1209, 1209, 1209, 572: 1209, 1209, 575: 1209, 577: 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 1209, 636: 1209}, // 990 - {2: 1206, 1206, 1206, 1206, 1206, 8: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 51: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 462: 1206, 464: 1206, 1206, 1206, 468: 1206, 471: 1206, 1206, 474: 1206, 1206, 1206, 481: 1206, 491: 1206, 494: 1206, 1206, 1206, 532: 1206, 555: 1206, 1206, 1206, 1206, 560: 1206, 1206, 1206, 1206, 1206, 1206, 567: 1206, 1206, 1206, 1206, 572: 1206, 1206, 575: 1206, 577: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 638: 1206}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3450}, - {50: 3451, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3452}, - {1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 463: 1093, 1093, 1093, 1093, 1093, 469: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 482: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 492: 1093, 1093, 497: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 533: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 571: 1093}, + {2: 1206, 1206, 1206, 1206, 1206, 8: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 51: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 462: 1206, 464: 1206, 1206, 1206, 468: 1206, 471: 1206, 1206, 474: 1206, 1206, 1206, 481: 1206, 491: 1206, 495: 1206, 1206, 1206, 534: 1206, 555: 1206, 1206, 1206, 1206, 560: 1206, 1206, 1206, 1206, 1206, 1206, 567: 1206, 1206, 1206, 1206, 572: 1206, 1206, 575: 1206, 577: 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 1206, 636: 1206}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3451}, + {50: 3452, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3453}, + {1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 463: 1093, 1093, 1093, 1093, 1093, 469: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 482: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 492: 1093, 1093, 1093, 498: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 535: 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 1093, 571: 1093}, // 995 - {930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 463: 930, 930, 930, 930, 930, 469: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 482: 930, 930, 930, 930, 930, 930, 930, 930, 930, 492: 930, 930, 497: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 533: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 571: 930}, - {2: 1205, 1205, 1205, 1205, 1205, 8: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 51: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 462: 1205, 464: 1205, 1205, 1205, 468: 1205, 471: 1205, 1205, 474: 1205, 1205, 1205, 481: 1205, 491: 1205, 494: 1205, 1205, 1205, 532: 1205, 555: 1205, 1205, 1205, 1205, 560: 1205, 1205, 1205, 1205, 1205, 1205, 567: 1205, 1205, 1205, 1205, 572: 1205, 1205, 575: 1205, 577: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 638: 1205}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3456, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3457}, - {50: 3458, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 463: 930, 930, 930, 930, 930, 469: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 482: 930, 930, 930, 930, 930, 930, 930, 930, 930, 492: 930, 930, 930, 498: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 535: 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 930, 571: 930}, + {2: 1205, 1205, 1205, 1205, 1205, 8: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 51: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 462: 1205, 464: 1205, 1205, 1205, 468: 1205, 471: 1205, 1205, 474: 1205, 1205, 1205, 481: 1205, 491: 1205, 495: 1205, 1205, 1205, 534: 1205, 555: 1205, 1205, 1205, 1205, 560: 1205, 1205, 1205, 1205, 1205, 1205, 567: 1205, 1205, 1205, 1205, 572: 1205, 1205, 575: 1205, 577: 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 1205, 636: 1205}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3457, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3458}, + {50: 3459, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1000 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3459}, - {1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 463: 1094, 1094, 1094, 1094, 1094, 469: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 482: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 492: 1094, 1094, 497: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 533: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 571: 1094}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3461, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3462}, - {50: 3463, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3460}, + {1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 463: 1094, 1094, 1094, 1094, 1094, 469: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 482: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 492: 1094, 1094, 1094, 498: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 535: 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 1094, 571: 1094}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3462, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3463}, + {50: 3464, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1005 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3464}, - {1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 463: 1095, 1095, 1095, 1095, 1095, 469: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 482: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 492: 1095, 1095, 497: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 533: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 571: 1095}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3466, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3467}, - {50: 3468, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3465}, + {1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 463: 1095, 1095, 1095, 1095, 1095, 469: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 482: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 492: 1095, 1095, 1095, 498: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 535: 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 1095, 571: 1095}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3467, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3468}, + {50: 3469, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1010 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3469}, - {1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 463: 1096, 1096, 1096, 1096, 1096, 469: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 482: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 492: 1096, 1096, 497: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 533: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 571: 1096}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3471, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3472}, - {50: 3473, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3470}, + {1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 463: 1096, 1096, 1096, 1096, 1096, 469: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 482: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 492: 1096, 1096, 1096, 498: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 535: 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 1096, 571: 1096}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3472, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3473}, + {50: 3474, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1015 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3474}, - {1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 463: 1097, 1097, 1097, 1097, 1097, 469: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 482: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 492: 1097, 1097, 497: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 533: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 571: 1097}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3476, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3477}, - {50: 3478, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3475}, + {1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 463: 1097, 1097, 1097, 1097, 1097, 469: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 482: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 492: 1097, 1097, 1097, 498: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 535: 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 1097, 571: 1097}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3477, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3478}, + {50: 3479, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1020 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3479}, - {1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 463: 1098, 1098, 1098, 1098, 1098, 469: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 482: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 492: 1098, 1098, 497: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 533: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 571: 1098}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3481, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3482}, - {50: 3483, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3480}, + {1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 463: 1098, 1098, 1098, 1098, 1098, 469: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 482: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 492: 1098, 1098, 1098, 498: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 535: 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 1098, 571: 1098}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3482, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3483}, + {50: 3484, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1025 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3484}, - {1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 463: 1099, 1099, 1099, 1099, 1099, 469: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 482: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 492: 1099, 1099, 497: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 533: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 571: 1099}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3486, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3488}, - {1880, 1880, 7: 1880, 50: 1880, 124: 1880, 470: 1880, 490: 1880, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3485}, + {1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 463: 1099, 1099, 1099, 1099, 1099, 469: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 482: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 492: 1099, 1099, 1099, 498: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 535: 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 1099, 571: 1099}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3487, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3489}, + {1880, 1880, 7: 1880, 50: 1880, 124: 1880, 470: 1880, 490: 1880, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1030 - {7: 3489, 50: 1265, 124: 1265, 490: 2637, 756: 2638, 799: 3490}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3496}, - {50: 1086, 124: 3492, 1240: 3491}, - {50: 3494}, - {464: 3493}, + {7: 3490, 50: 1265, 124: 1265, 490: 2638, 754: 2639, 797: 3491}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3497}, + {50: 1086, 124: 3493, 1238: 3492}, + {50: 3495}, + {464: 3494}, // 1035 {50: 1085}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3495}, - {1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 463: 1100, 1100, 1100, 1100, 1100, 469: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 482: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 492: 1100, 1100, 497: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 533: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 571: 1100}, - {1879, 1879, 7: 1879, 50: 1879, 124: 1879, 470: 1879, 490: 1879, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 512: 3501, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3500, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3498, 735: 3443, 3444, 772: 3499}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3496}, + {1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 463: 1100, 1100, 1100, 1100, 1100, 469: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 482: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 492: 1100, 1100, 1100, 498: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 535: 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 1100, 571: 1100}, + {1879, 1879, 7: 1879, 50: 1879, 124: 1879, 470: 1879, 490: 1879, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 512: 3502, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3501, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3499, 733: 3444, 3445, 770: 3500}, // 1040 - {50: 3509, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3507}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3504}, - {50: 3502}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3503}, + {50: 3510, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3508}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3505}, + {50: 3503}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3504}, // 1045 - {1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 463: 1101, 1101, 1101, 1101, 1101, 469: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 482: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 492: 1101, 1101, 497: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 533: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 571: 1101}, - {50: 3505, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3506}, - {1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 463: 1103, 1103, 1103, 1103, 1103, 469: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 482: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 492: 1103, 1103, 497: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 533: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 571: 1103}, - {7: 3489, 50: 3508}, + {1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 463: 1101, 1101, 1101, 1101, 1101, 469: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 482: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 492: 1101, 1101, 1101, 498: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 535: 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 1101, 571: 1101}, + {50: 3506, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3507}, + {1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 463: 1103, 1103, 1103, 1103, 1103, 469: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 482: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 492: 1103, 1103, 1103, 498: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 535: 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 1103, 571: 1103}, + {7: 3490, 50: 3509}, // 1050 - {1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 463: 1104, 1104, 1104, 1104, 1104, 469: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 482: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 492: 1104, 1104, 497: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 533: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 571: 1104}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3510}, - {1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 463: 1102, 1102, 1102, 1102, 1102, 469: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 482: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 492: 1102, 1102, 497: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 533: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 571: 1102}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3513, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3512}, - {50: 3517, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 463: 1104, 1104, 1104, 1104, 1104, 469: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 482: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 492: 1104, 1104, 1104, 498: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 535: 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 1104, 571: 1104}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3511}, + {1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 463: 1102, 1102, 1102, 1102, 1102, 469: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 482: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 492: 1102, 1102, 1102, 498: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 535: 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 1102, 571: 1102}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3514, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3513}, + {50: 3518, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1055 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3514}, - {50: 3515, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3516}, - {1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 463: 1105, 1105, 1105, 1105, 1105, 469: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 482: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 492: 1105, 1105, 497: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 533: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 571: 1105}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3518}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3515}, + {50: 3516, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3517}, + {1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 463: 1105, 1105, 1105, 1105, 1105, 469: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 482: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 492: 1105, 1105, 1105, 498: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 535: 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 1105, 571: 1105}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3519}, // 1060 - {1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 463: 1106, 1106, 1106, 1106, 1106, 469: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 482: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 492: 1106, 1106, 497: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 533: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 571: 1106}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3521, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3520}, - {50: 3525, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3522}, - {50: 3523, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 463: 1106, 1106, 1106, 1106, 1106, 469: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 482: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 492: 1106, 1106, 1106, 498: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 535: 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 1106, 571: 1106}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3522, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3521}, + {50: 3526, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3523}, + {50: 3524, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1065 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3524}, - {1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 463: 1107, 1107, 1107, 1107, 1107, 469: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 482: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 492: 1107, 1107, 497: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 533: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 571: 1107}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3526}, - {1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 463: 1108, 1108, 1108, 1108, 1108, 469: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 482: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 492: 1108, 1108, 497: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 533: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 571: 1108}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3529, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3528}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3525}, + {1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 463: 1107, 1107, 1107, 1107, 1107, 469: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 482: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 492: 1107, 1107, 1107, 498: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 535: 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 1107, 571: 1107}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3527}, + {1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 463: 1108, 1108, 1108, 1108, 1108, 469: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 482: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 492: 1108, 1108, 1108, 498: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 535: 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 1108, 571: 1108}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3530, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3529}, // 1070 - {50: 3533, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3530}, - {50: 3531, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3532}, - {1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 463: 1109, 1109, 1109, 1109, 1109, 469: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 482: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 492: 1109, 1109, 497: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 533: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 571: 1109}, + {50: 3534, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3531}, + {50: 3532, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3533}, + {1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 463: 1109, 1109, 1109, 1109, 1109, 469: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 482: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 492: 1109, 1109, 1109, 498: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 535: 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 1109, 571: 1109}, // 1075 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3534}, - {1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 463: 1110, 1110, 1110, 1110, 1110, 469: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 482: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 492: 1110, 1110, 497: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 533: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 571: 1110}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3536}, - {7: 3489, 50: 3537}, - {1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 463: 1111, 1111, 1111, 1111, 1111, 469: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 482: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 492: 1111, 1111, 497: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 533: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 571: 1111}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3535}, + {1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 463: 1110, 1110, 1110, 1110, 1110, 469: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 482: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 492: 1110, 1110, 1110, 498: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 535: 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 1110, 571: 1110}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3537}, + {7: 3490, 50: 3538}, + {1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 463: 1111, 1111, 1111, 1111, 1111, 469: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 482: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 492: 1111, 1111, 1111, 498: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 535: 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 1111, 571: 1111}, // 1080 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3539}, - {7: 3489, 50: 3540}, - {1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 463: 1112, 1112, 1112, 1112, 1112, 469: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 482: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 492: 1112, 1112, 497: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 533: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 571: 1112}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3542}, - {7: 3543, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3540}, + {7: 3490, 50: 3541}, + {1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 463: 1112, 1112, 1112, 1112, 1112, 469: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 482: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 492: 1112, 1112, 1112, 498: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 535: 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 1112, 571: 1112}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3543}, + {7: 3544, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1085 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3544}, - {7: 3545, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3546}, - {50: 3547, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 463: 1128, 1128, 1128, 1128, 1128, 469: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 482: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 492: 1128, 1128, 497: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 533: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 571: 1128}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3545}, + {7: 3546, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3547}, + {50: 3548, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 463: 1128, 1128, 1128, 1128, 1128, 469: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 482: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 492: 1128, 1128, 1128, 498: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 535: 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 1128, 571: 1128}, // 1090 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3549, 1161: 3551, 1216: 3552, 1301: 3553, 3550}, - {50: 3561, 486: 3562, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 486: 3555, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3554}, - {2: 1119, 1119, 1119, 1119, 1119, 8: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 51: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 462: 1119, 464: 1119, 1119, 1119, 468: 1119, 471: 1119, 1119, 474: 1119, 1119, 1119, 481: 1119, 486: 1119, 491: 1119, 494: 1119, 1119, 1119, 532: 1119, 555: 1119, 1119, 1119, 1119, 560: 1119, 1119, 1119, 1119, 1119, 1119, 567: 1119, 1119, 1119, 1119, 572: 1119, 1119, 575: 1119, 577: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 638: 1119}, - {2: 1118, 1118, 1118, 1118, 1118, 8: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 51: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 462: 1118, 464: 1118, 1118, 1118, 468: 1118, 471: 1118, 1118, 474: 1118, 1118, 1118, 481: 1118, 486: 1118, 491: 1118, 494: 1118, 1118, 1118, 532: 1118, 555: 1118, 1118, 1118, 1118, 560: 1118, 1118, 1118, 1118, 1118, 1118, 567: 1118, 1118, 1118, 1118, 572: 1118, 1118, 575: 1118, 577: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 638: 1118}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3550, 1160: 3552, 1215: 3553, 1299: 3554, 3551}, + {50: 3562, 486: 3563, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 486: 3556, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3555}, + {2: 1119, 1119, 1119, 1119, 1119, 8: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 51: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 462: 1119, 464: 1119, 1119, 1119, 468: 1119, 471: 1119, 1119, 474: 1119, 1119, 1119, 481: 1119, 486: 1119, 491: 1119, 495: 1119, 1119, 1119, 534: 1119, 555: 1119, 1119, 1119, 1119, 560: 1119, 1119, 1119, 1119, 1119, 1119, 567: 1119, 1119, 1119, 1119, 572: 1119, 1119, 575: 1119, 577: 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 1119, 636: 1119}, + {2: 1118, 1118, 1118, 1118, 1118, 8: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 51: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 462: 1118, 464: 1118, 1118, 1118, 468: 1118, 471: 1118, 1118, 474: 1118, 1118, 1118, 481: 1118, 486: 1118, 491: 1118, 495: 1118, 1118, 1118, 534: 1118, 555: 1118, 1118, 1118, 1118, 560: 1118, 1118, 1118, 1118, 1118, 1118, 567: 1118, 1118, 1118, 1118, 572: 1118, 1118, 575: 1118, 577: 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 1118, 636: 1118}, // 1095 - {2: 1117, 1117, 1117, 1117, 1117, 8: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 51: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 462: 1117, 464: 1117, 1117, 1117, 468: 1117, 471: 1117, 1117, 474: 1117, 1117, 1117, 481: 1117, 486: 1117, 491: 1117, 494: 1117, 1117, 1117, 532: 1117, 555: 1117, 1117, 1117, 1117, 560: 1117, 1117, 1117, 1117, 1117, 1117, 567: 1117, 1117, 1117, 1117, 572: 1117, 1117, 575: 1117, 577: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 638: 1117}, - {486: 3558, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3556}, - {50: 3557, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 463: 1134, 1134, 1134, 1134, 1134, 469: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 482: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 492: 1134, 1134, 497: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 533: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 571: 1134}, + {2: 1117, 1117, 1117, 1117, 1117, 8: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 51: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 462: 1117, 464: 1117, 1117, 1117, 468: 1117, 471: 1117, 1117, 474: 1117, 1117, 1117, 481: 1117, 486: 1117, 491: 1117, 495: 1117, 1117, 1117, 534: 1117, 555: 1117, 1117, 1117, 1117, 560: 1117, 1117, 1117, 1117, 1117, 1117, 567: 1117, 1117, 1117, 1117, 572: 1117, 1117, 575: 1117, 577: 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 1117, 636: 1117}, + {486: 3559, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3557}, + {50: 3558, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 463: 1134, 1134, 1134, 1134, 1134, 469: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 482: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 492: 1134, 1134, 1134, 498: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 535: 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 1134, 571: 1134}, // 1100 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3559}, - {50: 3560, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 463: 1133, 1133, 1133, 1133, 1133, 469: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 482: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 492: 1133, 1133, 497: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 533: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 571: 1133}, - {1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 463: 1136, 1136, 1136, 1136, 1136, 469: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 482: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 492: 1136, 1136, 497: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 533: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 571: 1136}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3563}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3560}, + {50: 3561, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 463: 1133, 1133, 1133, 1133, 1133, 469: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 482: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 492: 1133, 1133, 1133, 498: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 535: 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 1133, 571: 1133}, + {1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 463: 1136, 1136, 1136, 1136, 1136, 469: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 482: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 492: 1136, 1136, 1136, 498: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 535: 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 1136, 571: 1136}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3564}, // 1105 - {50: 3564, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 463: 1135, 1135, 1135, 1135, 1135, 469: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 482: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 492: 1135, 1135, 497: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 533: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 571: 1135}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3566}, - {7: 3567, 486: 3568, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3574}, + {50: 3565, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 463: 1135, 1135, 1135, 1135, 1135, 469: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 482: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 492: 1135, 1135, 1135, 498: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 535: 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 1135, 571: 1135}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3567}, + {7: 3568, 486: 3569, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3575}, // 1110 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3569}, - {50: 3570, 482: 3571, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 463: 1141, 1141, 1141, 1141, 1141, 469: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 482: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 492: 1141, 1141, 497: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 533: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 571: 1141}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3572}, - {50: 3573, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3570}, + {50: 3571, 482: 3572, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 463: 1141, 1141, 1141, 1141, 1141, 469: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 482: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 492: 1141, 1141, 1141, 498: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 535: 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 1141, 571: 1141}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3573}, + {50: 3574, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1115 - {1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 463: 1139, 1139, 1139, 1139, 1139, 469: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 482: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 492: 1139, 1139, 497: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 533: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 571: 1139}, - {7: 3576, 50: 3575, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 463: 1142, 1142, 1142, 1142, 1142, 469: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 482: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 492: 1142, 1142, 497: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 533: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 571: 1142}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3577}, - {50: 3578, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 463: 1139, 1139, 1139, 1139, 1139, 469: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 482: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 492: 1139, 1139, 1139, 498: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 535: 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 1139, 571: 1139}, + {7: 3577, 50: 3576, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 463: 1142, 1142, 1142, 1142, 1142, 469: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 482: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 492: 1142, 1142, 1142, 498: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 535: 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 1142, 571: 1142}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3578}, + {50: 3579, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1120 - {1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 463: 1140, 1140, 1140, 1140, 1140, 469: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 482: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 492: 1140, 1140, 497: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 533: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 571: 1140}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3580}, - {474: 3586, 3585, 3591, 512: 3587, 533: 3593, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3613}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3612}, + {1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 463: 1140, 1140, 1140, 1140, 1140, 469: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 482: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 492: 1140, 1140, 1140, 498: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 535: 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 1140, 571: 1140}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3581}, + {474: 3587, 3586, 3592, 512: 3588, 532: 3594, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3614}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3613}, // 1125 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3611}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3610}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3607, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3606}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3603, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3602}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3601}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3612}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3611}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3608, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3607}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3604, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3603}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3602}, // 1130 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3600}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3599}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3598}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3597}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3596}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3601}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3600}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3599}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3598}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3597}, // 1135 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3594}, - {50: 3595, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 463: 1143, 1143, 1143, 1143, 1143, 469: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 482: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 492: 1143, 1143, 497: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 533: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 571: 1143}, - {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 463: 1250, 1250, 1250, 1250, 1250, 469: 1250, 1250, 1250, 1250, 474: 1250, 1250, 1250, 1250, 1250, 1250, 482: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 492: 1250, 1250, 497: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 533: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250}, - {1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 463: 1251, 1251, 1251, 1251, 1251, 469: 1251, 1251, 1251, 1251, 474: 1251, 1251, 1251, 1251, 1251, 1251, 482: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 492: 1251, 1251, 497: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 533: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 3592, 1251, 1251, 1251, 1251, 1251, 1251}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3595}, + {50: 3596, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 463: 1143, 1143, 1143, 1143, 1143, 469: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 482: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 492: 1143, 1143, 1143, 498: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 535: 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 1143, 571: 1143}, + {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 463: 1250, 1250, 1250, 1250, 1250, 469: 1250, 1250, 1250, 1250, 474: 1250, 1250, 1250, 1250, 1250, 1250, 482: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 492: 1250, 1250, 1250, 498: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 535: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250}, + {1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 463: 1251, 1251, 1251, 1251, 1251, 469: 1251, 1251, 1251, 1251, 474: 1251, 1251, 1251, 1251, 1251, 1251, 482: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 492: 1251, 1251, 1251, 498: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 535: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 3593, 1251, 1251, 1251, 1251, 1251, 1251}, // 1140 - {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 463: 1252, 1252, 1252, 1252, 1252, 469: 1252, 1252, 1252, 1252, 474: 1252, 1252, 1252, 1252, 1252, 1252, 482: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 492: 1252, 1252, 497: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 533: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3592, 1252, 1252, 1252, 1252, 1252, 1252}, - {1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 463: 1253, 1253, 1253, 1253, 1253, 469: 1253, 1253, 1253, 1253, 474: 1253, 1253, 1253, 1253, 1253, 1253, 482: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 492: 1253, 1253, 497: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 533: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3592, 1253, 1253, 1253, 1253, 1253, 1253}, - {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 463: 1254, 1254, 1254, 1254, 1254, 469: 1254, 1254, 1254, 1254, 474: 1254, 1254, 1254, 1254, 1254, 1254, 482: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 492: 1254, 1254, 497: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 533: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3592, 1254, 1254, 1254, 1254, 1254, 1254}, - {1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 463: 1255, 1255, 1255, 1255, 1255, 469: 1255, 1255, 1255, 1255, 474: 1255, 1255, 1255, 1255, 1255, 1255, 482: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 492: 1255, 1255, 497: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 533: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 3592, 1255, 1255, 1255, 1255, 1255, 1255}, - {1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 463: 1258, 1258, 1258, 1258, 1258, 469: 1258, 1258, 1258, 1258, 474: 1258, 1258, 3591, 1258, 1258, 1258, 482: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 492: 1258, 1258, 497: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 3587, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 533: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 3588, 3589, 1258, 3592, 1258, 3590, 1258, 1258, 1258, 1258}, + {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 463: 1252, 1252, 1252, 1252, 1252, 469: 1252, 1252, 1252, 1252, 474: 1252, 1252, 1252, 1252, 1252, 1252, 482: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 492: 1252, 1252, 1252, 498: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 535: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3593, 1252, 1252, 1252, 1252, 1252, 1252}, + {1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 463: 1253, 1253, 1253, 1253, 1253, 469: 1253, 1253, 1253, 1253, 474: 1253, 1253, 1253, 1253, 1253, 1253, 482: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 492: 1253, 1253, 1253, 498: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 535: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3593, 1253, 1253, 1253, 1253, 1253, 1253}, + {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 463: 1254, 1254, 1254, 1254, 1254, 469: 1254, 1254, 1254, 1254, 474: 1254, 1254, 1254, 1254, 1254, 1254, 482: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 492: 1254, 1254, 1254, 498: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 535: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3593, 1254, 1254, 1254, 1254, 1254, 1254}, + {1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 463: 1255, 1255, 1255, 1255, 1255, 469: 1255, 1255, 1255, 1255, 474: 1255, 1255, 1255, 1255, 1255, 1255, 482: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 492: 1255, 1255, 1255, 498: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 535: 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 1255, 3593, 1255, 1255, 1255, 1255, 1255, 1255}, + {1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 463: 1258, 1258, 1258, 1258, 1258, 469: 1258, 1258, 1258, 1258, 474: 1258, 1258, 3592, 1258, 1258, 1258, 482: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 492: 1258, 1258, 1258, 498: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 3588, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 535: 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 1258, 3589, 3590, 1258, 3593, 1258, 3591, 1258, 1258, 1258, 1258}, // 1145 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 1195, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3604}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3605}, - {1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 463: 1256, 1256, 1256, 1256, 1256, 469: 1256, 1256, 1256, 1256, 474: 1256, 1256, 1256, 1256, 1256, 1256, 482: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 492: 1256, 1256, 497: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 533: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256}, - {1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 463: 1259, 1259, 1259, 1259, 1259, 469: 1259, 1259, 1259, 1259, 474: 1259, 1259, 3591, 1259, 1259, 1259, 482: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 492: 1259, 1259, 497: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 3587, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 533: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 3588, 3589, 1259, 3592, 1259, 3590, 1259, 1259, 1259, 1259}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 1195, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3608}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 1195, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3605}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3606}, + {1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 463: 1256, 1256, 1256, 1256, 1256, 469: 1256, 1256, 1256, 1256, 474: 1256, 1256, 1256, 1256, 1256, 1256, 482: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 492: 1256, 1256, 1256, 498: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 535: 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256, 1256}, + {1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 463: 1259, 1259, 1259, 1259, 1259, 469: 1259, 1259, 1259, 1259, 474: 1259, 1259, 3592, 1259, 1259, 1259, 482: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 492: 1259, 1259, 1259, 498: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 3588, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 535: 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 1259, 3589, 3590, 1259, 3593, 1259, 3591, 1259, 1259, 1259, 1259}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 1195, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3609}, // 1150 - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3609}, - {1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 463: 1257, 1257, 1257, 1257, 1257, 469: 1257, 1257, 1257, 1257, 474: 1257, 1257, 1257, 1257, 1257, 1257, 482: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 492: 1257, 1257, 497: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 533: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257}, - {1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 463: 1260, 1260, 1260, 1260, 1260, 469: 1260, 1260, 1260, 1260, 474: 3586, 3585, 3591, 1260, 1260, 1260, 482: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 492: 1260, 1260, 497: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 3587, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 533: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 3588, 3589, 1260, 3592, 1260, 3590, 1260, 1260, 1260, 1260}, - {1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 463: 1261, 1261, 1261, 1261, 1261, 469: 1261, 1261, 1261, 1261, 474: 3586, 3585, 3591, 1261, 1261, 1261, 482: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 492: 1261, 1261, 497: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 3587, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 533: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 3588, 3589, 1261, 3592, 1261, 3590, 1261, 1261, 1261, 1261}, - {1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 463: 1262, 1262, 1262, 1262, 1262, 469: 1262, 1262, 1262, 1262, 474: 3586, 3585, 3591, 1262, 1262, 1262, 482: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 492: 1262, 1262, 497: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 3587, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 533: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 3588, 3589, 1262, 3592, 1262, 3590, 3583, 3584, 1262, 1262}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3610}, + {1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 463: 1257, 1257, 1257, 1257, 1257, 469: 1257, 1257, 1257, 1257, 474: 1257, 1257, 1257, 1257, 1257, 1257, 482: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 492: 1257, 1257, 1257, 498: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 535: 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257, 1257}, + {1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 463: 1260, 1260, 1260, 1260, 1260, 469: 1260, 1260, 1260, 1260, 474: 3587, 3586, 3592, 1260, 1260, 1260, 482: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 492: 1260, 1260, 1260, 498: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 3588, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 535: 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 1260, 3589, 3590, 1260, 3593, 1260, 3591, 1260, 1260, 1260, 1260}, + {1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 463: 1261, 1261, 1261, 1261, 1261, 469: 1261, 1261, 1261, 1261, 474: 3587, 3586, 3592, 1261, 1261, 1261, 482: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 492: 1261, 1261, 1261, 498: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 3588, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 535: 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 1261, 3589, 3590, 1261, 3593, 1261, 3591, 1261, 1261, 1261, 1261}, + {1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 463: 1262, 1262, 1262, 1262, 1262, 469: 1262, 1262, 1262, 1262, 474: 3587, 3586, 3592, 1262, 1262, 1262, 482: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 492: 1262, 1262, 1262, 498: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 3588, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 535: 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 1262, 3589, 3590, 1262, 3593, 1262, 3591, 3584, 3585, 1262, 1262}, // 1155 - {1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 463: 1263, 1263, 1263, 1263, 1263, 469: 1263, 1263, 1263, 1263, 474: 3586, 3585, 3591, 1263, 1263, 1263, 482: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 492: 1263, 1263, 497: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 3587, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 533: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 3588, 3589, 3582, 3592, 1263, 3590, 3583, 3584, 1263, 1263}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 794: 3343, 817: 3615}, - {486: 3616}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3617}, - {50: 3618, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 463: 1263, 1263, 1263, 1263, 1263, 469: 1263, 1263, 1263, 1263, 474: 3587, 3586, 3592, 1263, 1263, 1263, 482: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 492: 1263, 1263, 1263, 498: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 3588, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 535: 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 1263, 3589, 3590, 3583, 3593, 1263, 3591, 3584, 3585, 1263, 1263}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 792: 3344, 815: 3616}, + {486: 3617}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3618}, + {50: 3619, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1160 - {1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 463: 1145, 1145, 1145, 1145, 1145, 469: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 482: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 492: 1145, 1145, 497: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 533: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 571: 1145}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3620}, - {7: 3621, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {570: 3622}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3623}, + {1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 463: 1145, 1145, 1145, 1145, 1145, 469: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 482: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 492: 1145, 1145, 1145, 498: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 535: 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 1145, 571: 1145}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3621}, + {7: 3622, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {570: 3623}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3624}, // 1165 - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3624}, - {50: 3625}, - {1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 463: 1146, 1146, 1146, 1146, 1146, 469: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 482: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 492: 1146, 1146, 497: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 533: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 571: 1146}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3627}, - {7: 3628, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3625}, + {50: 3626}, + {1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 463: 1146, 1146, 1146, 1146, 1146, 469: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 482: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 492: 1146, 1146, 1146, 498: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 535: 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 1146, 571: 1146}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3628}, + {7: 3629, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1170 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3630, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3629}, - {50: 3634, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 1195, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3631}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 3632}, - {50: 3633}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3631, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3630}, + {50: 3635, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 1195, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3632}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 3633}, + {50: 3634}, // 1175 - {1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 463: 1147, 1147, 1147, 1147, 1147, 469: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 482: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 492: 1147, 1147, 497: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 533: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 571: 1147}, - {1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 463: 1148, 1148, 1148, 1148, 1148, 469: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 482: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 492: 1148, 1148, 497: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 533: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 571: 1148}, - {50: 1874, 496: 3637, 1034: 3636, 3638}, + {1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 463: 1147, 1147, 1147, 1147, 1147, 469: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 482: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 492: 1147, 1147, 1147, 498: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 535: 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 1147, 571: 1147}, + {1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 463: 1148, 1148, 1148, 1148, 1148, 469: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 482: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 492: 1148, 1148, 1148, 498: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 535: 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 1148, 571: 1148}, + {50: 1874, 497: 3638, 1032: 3637, 3639}, {50: 1873}, {50: 1872}, // 1180 - {50: 3639}, - {1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 463: 1149, 1149, 1149, 1149, 1149, 469: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 482: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 492: 1149, 1149, 497: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 533: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 571: 1149}, - {50: 1874, 496: 3637, 1034: 3636, 3641}, - {50: 3642}, - {1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 463: 1150, 1150, 1150, 1150, 1150, 469: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 482: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 492: 1150, 1150, 497: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 533: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 571: 1150}, + {50: 3640}, + {1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 463: 1149, 1149, 1149, 1149, 1149, 469: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 482: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 492: 1149, 1149, 1149, 498: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 535: 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 1149, 571: 1149}, + {50: 1874, 497: 3638, 1032: 3637, 3642}, + {50: 3643}, + {1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 463: 1150, 1150, 1150, 1150, 1150, 469: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 482: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 492: 1150, 1150, 1150, 498: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 535: 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 1150, 571: 1150}, // 1185 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3644}, - {7: 3645, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 3646}, - {50: 3647, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 463: 1152, 1152, 1152, 1152, 1152, 469: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 482: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 492: 1152, 1152, 497: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 533: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 571: 1152}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3645}, + {7: 3646, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 3647}, + {50: 3648, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 463: 1152, 1152, 1152, 1152, 1152, 469: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 482: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 492: 1152, 1152, 1152, 498: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 535: 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 1152, 571: 1152}, // 1190 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3650}, - {7: 3489, 50: 1875}, - {50: 3651}, - {1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 463: 1153, 1153, 1153, 1153, 1153, 469: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 482: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 492: 1153, 1153, 497: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 533: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 571: 1153}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3653}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3651}, + {7: 3490, 50: 1875}, + {50: 3652}, + {1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 463: 1153, 1153, 1153, 1153, 1153, 469: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 482: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 492: 1153, 1153, 1153, 498: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 535: 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 1153, 571: 1153}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3654}, // 1195 - {7: 3489, 50: 3654, 470: 3655}, - {1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 463: 1158, 1158, 1158, 1158, 1158, 469: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 482: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 492: 1158, 1158, 497: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 533: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 571: 1158}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 3656}, - {50: 3659}, - {730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 50: 730, 98: 730, 107: 730, 461: 730, 730, 730, 465: 730, 730, 730, 730, 730, 473: 730, 479: 730, 730, 730, 491: 730, 730, 494: 730, 730, 501: 730, 532: 730, 559: 730, 566: 730, 574: 730, 576: 730, 632: 730, 730, 730, 730, 730, 730, 639: 730, 648: 730}, + {7: 3490, 50: 3655, 470: 3656}, + {1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 463: 1158, 1158, 1158, 1158, 1158, 469: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 482: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 492: 1158, 1158, 1158, 498: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 535: 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 1158, 571: 1158}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 3657}, + {50: 3660}, + {730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, 50: 730, 98: 730, 107: 730, 461: 730, 730, 730, 465: 730, 730, 730, 730, 730, 473: 730, 479: 730, 730, 730, 491: 730, 730, 730, 495: 730, 730, 534: 730, 559: 730, 566: 730, 574: 730, 576: 730, 630: 730, 730, 730, 730, 730, 730, 637: 730, 646: 730}, // 1200 - {729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 50: 729, 98: 729, 107: 729, 461: 729, 729, 729, 465: 729, 729, 729, 729, 729, 473: 729, 479: 729, 729, 729, 491: 729, 729, 494: 729, 729, 501: 729, 532: 729, 559: 729, 566: 729, 574: 729, 576: 729, 632: 729, 729, 729, 729, 729, 729, 639: 729, 648: 729}, - {1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 463: 1157, 1157, 1157, 1157, 1157, 469: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 482: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 492: 1157, 1157, 497: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 533: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 571: 1157}, - {1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 463: 1159, 1159, 1159, 1159, 1159, 469: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 482: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 492: 1159, 1159, 497: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 533: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 571: 1159}, - {50: 3662, 496: 3663}, - {1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 463: 1081, 1081, 1081, 1081, 1081, 469: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 482: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 492: 1081, 1081, 497: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 533: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 571: 1081}, + {729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 729, 50: 729, 98: 729, 107: 729, 461: 729, 729, 729, 465: 729, 729, 729, 729, 729, 473: 729, 479: 729, 729, 729, 491: 729, 729, 729, 495: 729, 729, 534: 729, 559: 729, 566: 729, 574: 729, 576: 729, 630: 729, 729, 729, 729, 729, 729, 637: 729, 646: 729}, + {1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 463: 1157, 1157, 1157, 1157, 1157, 469: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 482: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 492: 1157, 1157, 1157, 498: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 535: 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 1157, 571: 1157}, + {1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 463: 1159, 1159, 1159, 1159, 1159, 469: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 482: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 492: 1159, 1159, 1159, 498: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 535: 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 1159, 571: 1159}, + {50: 3663, 497: 3664}, + {1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 463: 1081, 1081, 1081, 1081, 1081, 469: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 482: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 492: 1081, 1081, 1081, 498: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 535: 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 1081, 571: 1081}, // 1205 - {50: 3664}, - {1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 463: 1080, 1080, 1080, 1080, 1080, 469: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 482: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 492: 1080, 1080, 497: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 533: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 571: 1080}, - {50: 3666}, - {1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 463: 1160, 1160, 1160, 1160, 1160, 469: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 482: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 492: 1160, 1160, 497: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 533: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 571: 1160}, - {50: 3669}, + {50: 3665}, + {1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 463: 1080, 1080, 1080, 1080, 1080, 469: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 482: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 492: 1080, 1080, 1080, 498: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 535: 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 1080, 571: 1080}, + {50: 3667}, + {1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 463: 1160, 1160, 1160, 1160, 1160, 469: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 482: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 492: 1160, 1160, 1160, 498: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 535: 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 1160, 571: 1160}, + {50: 3670}, // 1210 - {1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 463: 1161, 1161, 1161, 1161, 1161, 469: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 482: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 492: 1161, 1161, 497: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 533: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 571: 1161}, - {1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 463: 1174, 1174, 1174, 1174, 1174, 469: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 482: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 492: 1174, 1174, 497: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 533: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 571: 1174, 640: 1174, 651: 1174, 658: 1174}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3671}, - {50: 3672}, - {1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 463: 1162, 1162, 1162, 1162, 1162, 469: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 482: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 492: 1162, 1162, 497: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 533: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 571: 1162}, + {1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 463: 1161, 1161, 1161, 1161, 1161, 469: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 482: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 492: 1161, 1161, 1161, 498: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 535: 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 1161, 571: 1161}, + {1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 463: 1174, 1174, 1174, 1174, 1174, 469: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 482: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 492: 1174, 1174, 1174, 498: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 535: 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 1174, 571: 1174, 638: 1174, 649: 1174, 656: 1174}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3672}, + {50: 3673}, + {1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 463: 1162, 1162, 1162, 1162, 1162, 469: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 482: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 492: 1162, 1162, 1162, 498: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 535: 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 1162, 571: 1162}, // 1215 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3674}, - {50: 3675}, - {1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 463: 1163, 1163, 1163, 1163, 1163, 469: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 482: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 492: 1163, 1163, 497: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 533: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 571: 1163}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3677, 2674, 2675, 2673, 701: 3678}, - {50: 1248, 487: 1248, 642: 3680}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3675}, + {50: 3676}, + {1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 463: 1163, 1163, 1163, 1163, 1163, 469: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 482: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 492: 1163, 1163, 1163, 498: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 535: 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 1163, 571: 1163}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3678, 2675, 2676, 2674, 699: 3679}, + {50: 1248, 487: 1248, 640: 3681}, // 1220 - {50: 3679}, - {1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 463: 1217, 1217, 1217, 1217, 1217, 469: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 482: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 492: 1217, 1217, 497: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 533: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 571: 1217}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3681, 2674, 2675, 2673}, - {50: 1247, 487: 1247, 642: 3682}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3683, 2674, 2675, 2673}, + {50: 3680}, + {1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 463: 1217, 1217, 1217, 1217, 1217, 469: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 482: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 492: 1217, 1217, 1217, 498: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 535: 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 1217, 571: 1217}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3682, 2675, 2676, 2674}, + {50: 1247, 487: 1247, 640: 3683}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3684, 2675, 2676, 2674}, // 1225 - {1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 463: 1246, 1246, 1246, 1246, 1246, 469: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 482: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 492: 1246, 1246, 497: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 533: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 571: 1246, 645: 1246, 1246}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3677, 2674, 2675, 2673, 701: 3685}, - {50: 3686}, - {1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 463: 1218, 1218, 1218, 1218, 1218, 469: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 482: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 492: 1218, 1218, 497: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 533: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 571: 1218}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3688}, + {1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 463: 1246, 1246, 1246, 1246, 1246, 469: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 482: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 492: 1246, 1246, 1246, 498: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 535: 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 1246, 571: 1246, 643: 1246, 1246}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3678, 2675, 2676, 2674, 699: 3686}, + {50: 3687}, + {1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 463: 1218, 1218, 1218, 1218, 1218, 469: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 482: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 492: 1218, 1218, 1218, 498: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 535: 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 1218, 571: 1218}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3689}, // 1230 - {7: 3689, 470: 3690, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {48: 3701, 105: 3697, 171: 3698, 3696, 175: 3703, 189: 3700, 494: 3708, 532: 3694, 637: 3707, 670: 3699, 3704, 3705, 675: 3706, 729: 3702, 888: 3695, 982: 3693}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 3691}, - {50: 3692}, - {1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 463: 1219, 1219, 1219, 1219, 1219, 469: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 482: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 492: 1219, 1219, 497: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 533: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 571: 1219}, + {7: 3690, 470: 3691, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {48: 3702, 105: 3698, 171: 3699, 3697, 175: 3704, 190: 3701, 495: 3709, 534: 3695, 635: 3708, 668: 3700, 3705, 3706, 673: 3707, 727: 3703, 886: 3696, 980: 3694}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 3692}, + {50: 3693}, + {1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 463: 1219, 1219, 1219, 1219, 1219, 469: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 482: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 492: 1219, 1219, 1219, 498: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 535: 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 1219, 571: 1219}, // 1235 - {50: 3744}, - {50: 278, 462: 3723, 750: 3724, 774: 3743}, - {14: 278, 50: 278, 462: 3723, 494: 278, 532: 278, 637: 278, 750: 3724, 774: 3728}, + {50: 3745}, + {50: 278, 462: 3724, 748: 3725, 772: 3744}, + {14: 278, 50: 278, 462: 3724, 495: 278, 534: 278, 635: 278, 748: 3725, 772: 3729}, {50: 1041}, {50: 1040}, // 1240 - {50: 278, 462: 3723, 750: 3724, 774: 3727}, - {50: 271, 462: 3710, 750: 3711, 891: 3726, 896: 3712}, - {50: 278, 462: 3723, 750: 3724, 774: 3722}, - {50: 342, 673: 3719, 3720, 1072: 3721}, - {50: 342, 673: 3719, 3720, 1072: 3718}, + {50: 278, 462: 3724, 748: 3725, 772: 3728}, + {50: 271, 462: 3711, 748: 3712, 889: 3727, 894: 3713}, + {50: 278, 462: 3724, 748: 3725, 772: 3723}, + {50: 342, 671: 3720, 3721, 1071: 3722}, + {50: 342, 671: 3720, 3721, 1071: 3719}, // 1245 {50: 1034}, {50: 1033}, - {50: 271, 462: 3710, 750: 3711, 891: 3709, 896: 3712}, + {50: 271, 462: 3711, 748: 3712, 889: 3710, 894: 3713}, {50: 1031}, - {14: 316, 50: 316, 462: 316, 494: 316, 532: 316, 637: 316}, + {14: 316, 50: 316, 462: 316, 495: 316, 534: 316, 635: 316}, // 1250 - {14: 315, 50: 315, 462: 315, 494: 315, 532: 315, 637: 315}, + {14: 315, 50: 315, 462: 315, 495: 315, 534: 315, 635: 315}, {50: 1032}, - {496: 2648, 725: 2647, 734: 3713}, - {270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 48: 270, 50: 270, 461: 270, 465: 270, 270, 270, 270, 473: 270, 480: 270, 270, 566: 270, 574: 270, 576: 270, 632: 270, 270, 270, 270, 729: 270, 731: 270}, - {269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 48: 269, 50: 269, 461: 269, 465: 269, 269, 269, 269, 473: 269, 480: 269, 269, 566: 269, 574: 269, 576: 269, 632: 269, 269, 269, 269, 729: 269, 731: 269}, + {497: 2649, 721: 2648, 731: 3714}, + {270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 270, 48: 270, 50: 270, 461: 270, 465: 270, 270, 270, 270, 473: 270, 480: 270, 270, 566: 270, 574: 270, 576: 270, 630: 270, 270, 270, 270, 727: 270, 729: 270}, + {269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 269, 48: 269, 50: 269, 461: 269, 465: 269, 269, 269, 269, 473: 269, 480: 269, 269, 566: 269, 574: 269, 576: 269, 630: 269, 269, 269, 269, 727: 269, 729: 269}, // 1255 - {7: 3715, 50: 3714}, - {279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 14: 279, 48: 279, 50: 279, 101: 279, 279, 104: 279, 461: 279, 465: 279, 279, 279, 279, 473: 279, 480: 279, 279, 494: 279, 518: 279, 279, 532: 279, 566: 279, 574: 279, 576: 279, 632: 279, 279, 279, 279, 637: 279, 729: 279, 731: 279}, - {496: 2648, 725: 2647, 734: 3716}, - {50: 3717}, - {268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 48: 268, 50: 268, 461: 268, 465: 268, 268, 268, 268, 473: 268, 480: 268, 268, 566: 268, 574: 268, 576: 268, 632: 268, 268, 268, 268, 729: 268, 731: 268}, + {7: 3716, 50: 3715}, + {279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 279, 14: 279, 48: 279, 50: 279, 101: 279, 279, 104: 279, 461: 279, 465: 279, 279, 279, 279, 473: 279, 480: 279, 279, 495: 279, 518: 279, 279, 534: 279, 566: 279, 574: 279, 576: 279, 630: 279, 279, 279, 279, 635: 279, 727: 279, 729: 279}, + {497: 2649, 721: 2648, 731: 3717}, + {50: 3718}, + {268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 268, 48: 268, 50: 268, 461: 268, 465: 268, 268, 268, 268, 473: 268, 480: 268, 268, 566: 268, 574: 268, 576: 268, 630: 268, 268, 268, 268, 727: 268, 729: 268}, // 1260 {50: 1035}, {50: 341}, @@ -7537,847 +7532,847 @@ var ( {50: 1036}, {50: 1037}, // 1265 - {496: 2648, 725: 2647, 734: 3725}, - {277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 14: 277, 48: 277, 50: 277, 101: 277, 277, 104: 277, 461: 277, 465: 277, 277, 277, 277, 473: 277, 480: 277, 277, 494: 277, 518: 277, 277, 532: 277, 566: 277, 574: 277, 576: 277, 632: 277, 277, 277, 277, 637: 277, 729: 277, 731: 277}, - {50: 3714}, + {497: 2649, 721: 2648, 731: 3726}, + {277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 277, 14: 277, 48: 277, 50: 277, 101: 277, 277, 104: 277, 461: 277, 465: 277, 277, 277, 277, 473: 277, 480: 277, 277, 495: 277, 518: 277, 277, 534: 277, 566: 277, 574: 277, 576: 277, 630: 277, 277, 277, 277, 635: 277, 727: 277, 729: 277}, + {50: 3715}, {50: 1038}, {50: 1039}, // 1270 - {14: 3733, 50: 265, 494: 3734, 532: 3730, 637: 3732, 761: 3731, 788: 3729}, + {14: 3734, 50: 265, 495: 3735, 534: 3731, 635: 3733, 759: 3732, 786: 3730}, {50: 1042}, - {262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 14: 3733, 50: 262, 461: 262, 465: 262, 262, 262, 262, 473: 262, 480: 262, 262, 494: 3734, 566: 262, 574: 262, 576: 262, 632: 262, 262, 262, 262, 637: 3732, 761: 3741, 1236: 3740}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 3737}, - {501: 3736}, + {262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 262, 14: 3734, 50: 262, 461: 262, 465: 262, 262, 262, 262, 473: 262, 480: 262, 262, 495: 3735, 566: 262, 574: 262, 576: 262, 630: 262, 262, 262, 262, 635: 3733, 759: 3742, 1234: 3741}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 3738}, + {493: 3737}, // 1275 - {259, 259, 259, 259, 259, 259, 259, 8: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 51: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 464: 259, 468: 259, 487: 259, 489: 259, 510: 259, 532: 259}, - {501: 3735}, - {258, 258, 258, 258, 258, 258, 258, 8: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 51: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 464: 258, 468: 258, 487: 258, 489: 258, 510: 258, 532: 258}, - {260, 260, 260, 260, 260, 260, 260, 8: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 51: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 464: 260, 468: 260, 487: 260, 489: 260, 510: 260, 532: 260}, - {267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 50: 267, 461: 267, 465: 267, 267, 267, 267, 473: 267, 480: 267, 267, 532: 3738, 566: 267, 574: 267, 576: 267, 632: 267, 267, 267, 267, 1235: 3739}, + {259, 259, 259, 259, 259, 259, 259, 8: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 51: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 464: 259, 468: 259, 487: 259, 489: 259, 510: 259, 534: 259}, + {493: 3736}, + {258, 258, 258, 258, 258, 258, 258, 8: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 51: 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 258, 464: 258, 468: 258, 487: 258, 489: 258, 510: 258, 534: 258}, + {260, 260, 260, 260, 260, 260, 260, 8: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 51: 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 260, 464: 260, 468: 260, 487: 260, 489: 260, 510: 260, 534: 260}, + {267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 267, 50: 267, 461: 267, 465: 267, 267, 267, 267, 473: 267, 480: 267, 267, 534: 3739, 566: 267, 574: 267, 576: 267, 630: 267, 267, 267, 267, 1233: 3740}, // 1280 - {266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 50: 266, 461: 266, 465: 266, 266, 266, 266, 473: 266, 480: 266, 266, 566: 266, 574: 266, 576: 266, 632: 266, 266, 266, 266}, - {263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 50: 263, 461: 263, 465: 263, 263, 263, 263, 473: 263, 480: 263, 263, 566: 263, 574: 263, 576: 263, 632: 263, 263, 263, 263}, - {264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 50: 264, 461: 264, 465: 264, 264, 264, 264, 473: 264, 480: 264, 264, 566: 264, 574: 264, 576: 264, 632: 264, 264, 264, 264}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 3742}, - {261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 50: 261, 461: 261, 465: 261, 261, 261, 261, 473: 261, 480: 261, 261, 566: 261, 574: 261, 576: 261, 632: 261, 261, 261, 261}, + {266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 266, 50: 266, 461: 266, 465: 266, 266, 266, 266, 473: 266, 480: 266, 266, 566: 266, 574: 266, 576: 266, 630: 266, 266, 266, 266}, + {263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 263, 50: 263, 461: 263, 465: 263, 263, 263, 263, 473: 263, 480: 263, 263, 566: 263, 574: 263, 576: 263, 630: 263, 263, 263, 263}, + {264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 264, 50: 264, 461: 264, 465: 264, 264, 264, 264, 473: 264, 480: 264, 264, 566: 264, 574: 264, 576: 264, 630: 264, 264, 264, 264}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 3743}, + {261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 261, 50: 261, 461: 261, 465: 261, 261, 261, 261, 473: 261, 480: 261, 261, 566: 261, 574: 261, 576: 261, 630: 261, 261, 261, 261}, // 1285 {50: 1043}, - {1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 463: 1220, 1220, 1220, 1220, 1220, 469: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 482: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 492: 1220, 1220, 497: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 533: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 571: 1220}, - {493: 3247, 497: 3245, 3246, 3244, 3242, 531: 1049, 723: 3243, 3241}, - {531: 3749, 1138: 3748, 1317: 3747}, - {155: 1045, 531: 3749, 534: 3755, 1138: 3754, 1182: 3753}, + {1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 463: 1220, 1220, 1220, 1220, 1220, 469: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 482: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 492: 1220, 1220, 1220, 498: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 535: 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 1220, 571: 1220}, + {494: 3246, 498: 3244, 3245, 3243, 3241, 531: 1049, 722: 3242, 3240}, + {531: 3750, 1137: 3749, 1315: 3748}, + {155: 1045, 531: 3750, 533: 3756, 1137: 3755, 1181: 3754}, // 1290 - {155: 1048, 531: 1048, 534: 1048}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3750}, - {493: 3247, 497: 3245, 3246, 3244, 3242, 535: 3751, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3752}, - {155: 1046, 493: 3247, 497: 3245, 3246, 3244, 3242, 531: 1046, 534: 1046, 723: 3243, 3241}, + {155: 1048, 531: 1048, 533: 1048}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3751}, + {494: 3246, 498: 3244, 3245, 3243, 3241, 535: 3752, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3753}, + {155: 1046, 494: 3246, 498: 3244, 3245, 3243, 3241, 531: 1046, 533: 1046, 722: 3242, 3240}, // 1295 - {155: 3757}, - {155: 1047, 531: 1047, 534: 1047}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3756}, - {155: 1044, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 463: 1221, 1221, 1221, 1221, 1221, 469: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 482: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 492: 1221, 1221, 497: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 533: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 571: 1221}, + {155: 3758}, + {155: 1047, 531: 1047, 533: 1047}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3757}, + {155: 1044, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 463: 1221, 1221, 1221, 1221, 1221, 469: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 482: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 492: 1221, 1221, 1221, 498: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 535: 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 1221, 571: 1221}, // 1300 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3759}, - {467: 3760, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {48: 3701, 105: 3697, 171: 3698, 3696, 175: 3703, 189: 3700, 494: 3708, 532: 3694, 637: 3707, 670: 3699, 3704, 3705, 675: 3706, 729: 3702, 888: 3695, 982: 3761}, - {50: 3762}, - {1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 463: 1222, 1222, 1222, 1222, 1222, 469: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 482: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 492: 1222, 1222, 497: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 533: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 571: 1222}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3760}, + {467: 3761, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {48: 3702, 105: 3698, 171: 3699, 3697, 175: 3704, 190: 3701, 495: 3709, 534: 3695, 635: 3708, 668: 3700, 3705, 3706, 673: 3707, 727: 3703, 886: 3696, 980: 3762}, + {50: 3763}, + {1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 463: 1222, 1222, 1222, 1222, 1222, 469: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 482: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 492: 1222, 1222, 1222, 498: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 535: 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 1222, 571: 1222}, // 1305 - {1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 463: 1223, 1223, 1223, 1223, 1223, 469: 1223, 1223, 1223, 1223, 3251, 1223, 1223, 1223, 1223, 1223, 1223, 482: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 492: 1223, 1223, 497: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 533: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 571: 1223}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3765}, - {493: 3247, 497: 3245, 3246, 3244, 3242, 511: 3766, 723: 3243, 3241}, - {1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 463: 1224, 1224, 1224, 1224, 1224, 469: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 482: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 492: 1224, 1224, 497: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 533: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 571: 1224}, - {1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 463: 1225, 1225, 1225, 1225, 1225, 469: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 482: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 492: 1225, 1225, 497: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 533: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 571: 1225}, + {1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 463: 1223, 1223, 1223, 1223, 1223, 469: 1223, 1223, 1223, 1223, 3250, 1223, 1223, 1223, 1223, 1223, 1223, 482: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 492: 1223, 1223, 1223, 498: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 535: 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 1223, 571: 1223}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3766}, + {494: 3246, 498: 3244, 3245, 3243, 3241, 511: 3767, 722: 3242, 3240}, + {1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 463: 1224, 1224, 1224, 1224, 1224, 469: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 482: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 492: 1224, 1224, 1224, 498: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 535: 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 1224, 571: 1224}, + {1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 463: 1225, 1225, 1225, 1225, 1225, 469: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 482: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 492: 1225, 1225, 1225, 498: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 535: 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 1225, 571: 1225}, // 1310 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3769}, - {7: 3770}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3771}, - {7: 1879, 50: 3772, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 463: 1226, 1226, 1226, 1226, 1226, 469: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 482: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 492: 1226, 1226, 497: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 533: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 571: 1226}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3770}, + {7: 3771}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3772}, + {7: 1879, 50: 3773, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 463: 1226, 1226, 1226, 1226, 1226, 469: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 482: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 492: 1226, 1226, 1226, 498: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 535: 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 1226, 571: 1226}, // 1315 - {7: 1880, 50: 3875, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {7: 3872}, - {7: 1229, 50: 1229, 465: 1229, 1229, 469: 786, 473: 1229, 1229, 1229, 1229, 786, 786, 483: 2640, 487: 1229, 2641, 490: 2637, 493: 1229, 497: 1229, 1229, 1229, 1229, 510: 1229, 512: 1229, 533: 1229, 536: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 571: 1229, 756: 3788, 3789}, - {462: 3676, 564: 3793, 899: 3792, 960: 3791}, - {462: 2507, 491: 2505, 559: 2504, 636: 2500, 699: 3785, 741: 3784, 2501, 2502, 2503, 2512, 2510, 3786, 3787}, + {7: 1880, 50: 3876, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {7: 3873}, + {7: 1229, 50: 1229, 465: 1229, 1229, 469: 786, 473: 1229, 1229, 1229, 1229, 786, 786, 483: 2641, 487: 1229, 2642, 490: 2638, 494: 1229, 498: 1229, 1229, 1229, 1229, 510: 1229, 512: 1229, 532: 1229, 536: 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 1229, 571: 1229, 754: 3789, 3790}, + {462: 3677, 564: 3794, 897: 3793, 958: 3792}, + {462: 2508, 491: 2506, 559: 2505, 634: 2501, 697: 3786, 739: 3785, 2502, 2503, 2504, 2513, 2511, 3787, 3788}, // 1320 - {50: 3783, 469: 787, 477: 787, 787}, + {50: 3784, 469: 787, 477: 787, 787}, + {50: 3783}, {50: 3782}, - {50: 3781}, - {814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 469: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 482: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 497: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 533: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 559: 814, 571: 814, 636: 814, 644: 814, 732: 814}, - {815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 469: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 482: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 497: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 533: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 559: 815, 571: 815, 636: 815, 644: 815, 732: 815}, + {814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 469: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 482: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 498: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 535: 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 814, 559: 814, 571: 814, 634: 814, 642: 814, 730: 814}, + {815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 469: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 482: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 498: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 535: 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 815, 559: 815, 571: 815, 634: 815, 642: 815, 730: 815}, // 1325 - {816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 469: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 482: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 497: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 533: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 559: 816, 571: 816, 636: 816, 644: 816, 732: 816}, + {816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 469: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 482: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 498: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 535: 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 816, 559: 816, 571: 816, 634: 816, 642: 816, 730: 816}, {971, 971, 50: 971, 461: 971, 463: 971, 469: 787, 971, 477: 787, 787}, - {970, 970, 50: 970, 461: 970, 463: 970, 469: 786, 970, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {970, 970, 50: 970, 461: 970, 463: 970, 469: 786, 970, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, {799, 799, 50: 799, 461: 799, 463: 799, 470: 799}, {798, 798, 50: 798, 461: 798, 463: 798, 470: 798}, // 1330 - {792, 792, 50: 792, 461: 792, 463: 792, 470: 792, 483: 2640, 488: 2641, 757: 3790}, + {792, 792, 50: 792, 461: 792, 463: 792, 470: 792, 483: 2641, 488: 2642, 755: 3791}, {791, 791, 50: 791, 461: 791, 463: 791, 470: 791}, {790, 790, 50: 790, 461: 790, 463: 790, 470: 790}, - {1265, 1265, 7: 3805, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2637, 756: 2638, 799: 3804}, + {1265, 1265, 7: 3806, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2638, 754: 2639, 797: 3805}, {8, 8, 7: 8, 50: 8, 461: 8, 463: 8, 469: 8, 8, 477: 8, 8, 482: 8, 8, 8, 8, 488: 8, 490: 8}, // 1335 - {462: 3794, 827: 3795}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1305, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 3799, 1307: 3798, 3797}, + {462: 3795, 825: 3796}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1305, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 3800, 1305: 3799, 3798}, {6, 6, 7: 6, 50: 6, 461: 6, 463: 6, 469: 6, 6, 477: 6, 6, 482: 6, 6, 6, 6, 488: 6, 490: 6}, - {1301, 1301, 7: 1301, 50: 1301, 461: 1301, 470: 1301, 483: 1301, 489: 1301, 1301, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {50: 3803}, + {1301, 1301, 7: 1301, 50: 1301, 461: 1301, 470: 1301, 483: 1301, 489: 1301, 1301, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {50: 3804}, // 1340 - {7: 3801, 50: 1304}, + {7: 3802, 50: 1304}, {7: 1302, 50: 1302}, - {1300, 1300, 7: 1300, 50: 1300, 461: 1300, 3684, 470: 1300, 483: 1300, 489: 1300, 1300}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 3802}, + {1300, 1300, 7: 1300, 50: 1300, 461: 1300, 3685, 470: 1300, 483: 1300, 489: 1300, 1300}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 3803}, {7: 1303, 50: 1303}, // 1345 - {1306, 1306, 7: 1306, 50: 1306, 86: 1306, 461: 1306, 463: 1306, 469: 1306, 1306, 477: 1306, 1306, 482: 1306, 1306, 1306, 1306, 488: 1306, 490: 1306, 493: 1306}, - {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2640, 841, 841, 488: 2641, 757: 2642, 816: 3807}, - {564: 3793, 899: 3806}, + {1306, 1306, 7: 1306, 50: 1306, 86: 1306, 461: 1306, 463: 1306, 469: 1306, 1306, 477: 1306, 1306, 482: 1306, 1306, 1306, 1306, 488: 1306, 490: 1306, 494: 1306}, + {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2641, 841, 841, 488: 2642, 755: 2643, 814: 3808}, + {564: 3794, 897: 3807}, {7, 7, 7: 7, 50: 7, 461: 7, 463: 7, 469: 7, 7, 477: 7, 7, 482: 7, 7, 7, 7, 488: 7, 490: 7}, - {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3809, 484: 812, 3810, 874: 3808}, + {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3810, 484: 812, 3811, 872: 3809}, // 1350 - {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3835, 875: 3834}, - {273: 3815, 644: 3814}, - {533: 3811}, - {273: 3812}, - {203: 3813}, + {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3836, 873: 3835}, + {274: 3816, 642: 3815}, + {532: 3812}, + {274: 3813}, + {203: 3814}, // 1355 {804, 804, 50: 804, 461: 804, 463: 804, 469: 804, 804, 477: 804, 804, 484: 804}, - {803, 803, 50: 803, 137: 803, 149: 803, 169: 803, 461: 803, 463: 803, 469: 803, 803, 477: 803, 803, 484: 803, 1065: 3817, 3828}, - {803, 803, 50: 803, 137: 803, 149: 803, 461: 803, 463: 803, 469: 803, 803, 477: 803, 803, 484: 803, 1065: 3817, 3816}, - {810, 810, 50: 810, 137: 3826, 149: 3825, 461: 810, 463: 810, 469: 810, 810, 477: 810, 810, 484: 810}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 3820}, + {803, 803, 50: 803, 137: 803, 149: 803, 169: 803, 461: 803, 463: 803, 469: 803, 803, 477: 803, 803, 484: 803, 1064: 3818, 3829}, + {803, 803, 50: 803, 137: 803, 149: 803, 461: 803, 463: 803, 469: 803, 803, 477: 803, 803, 484: 803, 1064: 3818, 3817}, + {810, 810, 50: 810, 137: 3827, 149: 3826, 461: 810, 463: 810, 469: 810, 810, 477: 810, 810, 484: 810}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 3821}, // 1360 - {1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 533: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 559: 1025, 566: 1025, 571: 1025, 574: 1025, 1025, 1025, 632: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 3823, 1025, 1025, 647: 1025, 1025, 1025, 652: 1025, 659: 1025, 1025, 1025, 1025, 1025, 1025, 669: 1025, 676: 1025, 1025, 679: 1025, 694: 1025}, - {1023, 1023, 7: 1023, 50: 1023, 137: 1023, 149: 1023, 169: 1023, 461: 1023, 463: 1023, 469: 1023, 1023, 477: 1023, 1023, 484: 1023, 486: 1023, 640: 1023, 660: 1023, 662: 1023}, - {802, 802, 7: 3821, 50: 802, 137: 802, 149: 802, 169: 802, 461: 802, 463: 802, 469: 802, 802, 477: 802, 802, 484: 802}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3822}, - {1022, 1022, 7: 1022, 50: 1022, 137: 1022, 149: 1022, 159: 1022, 169: 1022, 461: 1022, 463: 1022, 469: 1022, 1022, 477: 1022, 1022, 484: 1022, 486: 1022, 640: 1022, 1022, 660: 1022, 662: 1022}, + {1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 535: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 559: 1025, 566: 1025, 571: 1025, 574: 1025, 1025, 1025, 630: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 3824, 1025, 1025, 645: 1025, 1025, 1025, 650: 1025, 657: 1025, 1025, 1025, 1025, 1025, 1025, 667: 1025, 674: 1025, 1025, 677: 1025, 692: 1025}, + {1023, 1023, 7: 1023, 50: 1023, 137: 1023, 149: 1023, 169: 1023, 461: 1023, 463: 1023, 469: 1023, 1023, 477: 1023, 1023, 484: 1023, 486: 1023, 638: 1023, 658: 1023, 660: 1023}, + {802, 802, 7: 3822, 50: 802, 137: 802, 149: 802, 169: 802, 461: 802, 463: 802, 469: 802, 802, 477: 802, 802, 484: 802}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3823}, + {1022, 1022, 7: 1022, 50: 1022, 137: 1022, 149: 1022, 159: 1022, 169: 1022, 461: 1022, 463: 1022, 469: 1022, 1022, 477: 1022, 1022, 484: 1022, 486: 1022, 638: 1022, 1022, 658: 1022, 660: 1022}, // 1365 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3824, 2674, 2675, 2673}, - {1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 533: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 559: 1024, 566: 1024, 571: 1024, 574: 1024, 1024, 1024, 632: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 643: 1024, 1024, 647: 1024, 1024, 1024, 652: 1024, 659: 1024, 1024, 1024, 1024, 1024, 1024, 669: 1024, 676: 1024, 1024, 679: 1024, 694: 1024}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3825, 2675, 2676, 2674}, + {1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 535: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 559: 1024, 566: 1024, 571: 1024, 574: 1024, 1024, 1024, 630: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 641: 1024, 1024, 645: 1024, 1024, 1024, 650: 1024, 657: 1024, 1024, 1024, 1024, 1024, 1024, 667: 1024, 674: 1024, 1024, 677: 1024, 692: 1024}, {807, 807, 50: 807, 461: 807, 463: 807, 469: 807, 807, 477: 807, 807, 484: 807}, - {255: 3827}, + {255: 3828}, {805, 805, 50: 805, 461: 805, 463: 805, 469: 805, 805, 477: 805, 805, 484: 805}, // 1370 - {811, 811, 50: 811, 137: 3831, 149: 3829, 169: 3830, 461: 811, 463: 811, 469: 811, 811, 477: 811, 811, 484: 811}, + {811, 811, 50: 811, 137: 3832, 149: 3830, 169: 3831, 461: 811, 463: 811, 469: 811, 811, 477: 811, 811, 484: 811}, {809, 809, 50: 809, 461: 809, 463: 809, 469: 809, 809, 477: 809, 809, 484: 809}, - {496: 2648, 725: 3833}, - {255: 3832}, + {497: 2649, 721: 3834}, + {255: 3833}, {806, 806, 50: 806, 461: 806, 463: 806, 469: 806, 806, 477: 806, 806, 484: 806}, // 1375 {808, 808, 50: 808, 461: 808, 463: 808, 469: 808, 808, 477: 808, 808, 484: 808}, {972, 972, 50: 972, 461: 972, 463: 972, 469: 972, 972, 477: 972, 972}, - {1247: 3836}, - {464: 3837}, - {94, 94, 50: 94, 98: 3841, 107: 3840, 461: 94, 463: 94, 469: 94, 94, 477: 94, 94, 648: 94, 821: 3839, 1031: 3838}, + {1245: 3837}, + {464: 3838}, + {94, 94, 50: 94, 98: 3842, 107: 3841, 461: 94, 463: 94, 469: 94, 94, 477: 94, 94, 646: 94, 819: 3840, 1029: 3839}, // 1380 - {81, 81, 50: 81, 461: 81, 463: 81, 469: 81, 81, 477: 81, 81, 648: 3862, 931: 3861}, - {770: 3844, 776: 3846, 782: 3847, 3845, 1030: 3843, 1189: 3842}, - {92, 92, 15: 92, 47: 92, 49: 92, 51: 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 461: 92, 92, 486: 92, 533: 92, 643: 92, 770: 92, 776: 92, 782: 92, 92}, - {91, 91, 15: 91, 47: 91, 49: 91, 51: 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 461: 91, 91, 486: 91, 533: 91, 643: 91, 770: 91, 776: 91, 782: 91, 91}, - {93, 93, 50: 93, 461: 93, 93, 93, 469: 93, 93, 477: 93, 93, 93, 501: 93, 648: 93, 770: 3844, 776: 3846, 782: 3847, 3845, 1030: 3860}, + {81, 81, 50: 81, 461: 81, 463: 81, 469: 81, 81, 477: 81, 81, 646: 3863, 929: 3862}, + {768: 3845, 774: 3847, 780: 3848, 3846, 1028: 3844, 1188: 3843}, + {92, 92, 15: 92, 47: 92, 49: 92, 51: 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 92, 461: 92, 92, 486: 92, 532: 92, 641: 92, 768: 92, 774: 92, 780: 92, 92}, + {91, 91, 15: 91, 47: 91, 49: 91, 51: 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 91, 461: 91, 91, 486: 91, 532: 91, 641: 91, 768: 91, 774: 91, 780: 91, 91}, + {93, 93, 50: 93, 461: 93, 93, 93, 469: 93, 93, 477: 93, 93, 93, 493: 93, 646: 93, 768: 3845, 774: 3847, 780: 3848, 3846, 1028: 3861}, // 1385 - {89, 89, 50: 89, 461: 89, 89, 89, 469: 89, 89, 477: 89, 89, 89, 501: 89, 648: 89, 770: 89, 776: 89, 782: 89, 89}, - {649: 3858}, - {776: 3855}, - {649: 3853}, - {649: 3848}, + {89, 89, 50: 89, 461: 89, 89, 89, 469: 89, 89, 477: 89, 89, 89, 493: 89, 646: 89, 768: 89, 774: 89, 780: 89, 89}, + {647: 3859}, + {774: 3856}, + {647: 3854}, + {647: 3849}, // 1390 - {464: 3850, 565: 3851, 569: 3852, 840: 3849}, - {85, 85, 50: 85, 461: 85, 85, 85, 469: 85, 85, 477: 85, 85, 85, 501: 85, 648: 85, 770: 85, 776: 85, 782: 85, 85}, - {84, 84, 50: 84, 461: 84, 84, 84, 469: 84, 84, 477: 84, 84, 84, 501: 84, 648: 84, 770: 84, 776: 84, 782: 84, 84}, - {83, 83, 50: 83, 461: 83, 83, 83, 469: 83, 83, 477: 83, 83, 83, 501: 83, 648: 83, 770: 83, 776: 83, 782: 83, 83}, - {82, 82, 50: 82, 461: 82, 82, 82, 469: 82, 82, 477: 82, 82, 82, 501: 82, 648: 82, 770: 82, 776: 82, 782: 82, 82}, + {464: 3851, 565: 3852, 569: 3853, 838: 3850}, + {85, 85, 50: 85, 461: 85, 85, 85, 469: 85, 85, 477: 85, 85, 85, 493: 85, 646: 85, 768: 85, 774: 85, 780: 85, 85}, + {84, 84, 50: 84, 461: 84, 84, 84, 469: 84, 84, 477: 84, 84, 84, 493: 84, 646: 84, 768: 84, 774: 84, 780: 84, 84}, + {83, 83, 50: 83, 461: 83, 83, 83, 469: 83, 83, 477: 83, 83, 83, 493: 83, 646: 83, 768: 83, 774: 83, 780: 83, 83}, + {82, 82, 50: 82, 461: 82, 82, 82, 469: 82, 82, 477: 82, 82, 82, 493: 82, 646: 82, 768: 82, 774: 82, 780: 82, 82}, // 1395 - {464: 3850, 565: 3851, 569: 3852, 840: 3854}, - {86, 86, 50: 86, 461: 86, 86, 86, 469: 86, 86, 477: 86, 86, 86, 501: 86, 648: 86, 770: 86, 776: 86, 782: 86, 86}, - {649: 3856}, - {464: 3850, 565: 3851, 569: 3852, 840: 3857}, - {87, 87, 50: 87, 461: 87, 87, 87, 469: 87, 87, 477: 87, 87, 87, 501: 87, 648: 87, 770: 87, 776: 87, 782: 87, 87}, + {464: 3851, 565: 3852, 569: 3853, 838: 3855}, + {86, 86, 50: 86, 461: 86, 86, 86, 469: 86, 86, 477: 86, 86, 86, 493: 86, 646: 86, 768: 86, 774: 86, 780: 86, 86}, + {647: 3857}, + {464: 3851, 565: 3852, 569: 3853, 838: 3858}, + {87, 87, 50: 87, 461: 87, 87, 87, 469: 87, 87, 477: 87, 87, 87, 493: 87, 646: 87, 768: 87, 774: 87, 780: 87, 87}, // 1400 - {464: 3850, 565: 3851, 569: 3852, 840: 3859}, - {88, 88, 50: 88, 461: 88, 88, 88, 469: 88, 88, 477: 88, 88, 88, 501: 88, 648: 88, 770: 88, 776: 88, 782: 88, 88}, - {90, 90, 50: 90, 461: 90, 90, 90, 469: 90, 90, 477: 90, 90, 90, 501: 90, 648: 90, 770: 90, 776: 90, 782: 90, 90}, + {464: 3851, 565: 3852, 569: 3853, 838: 3860}, + {88, 88, 50: 88, 461: 88, 88, 88, 469: 88, 88, 477: 88, 88, 88, 493: 88, 646: 88, 768: 88, 774: 88, 780: 88, 88}, + {90, 90, 50: 90, 461: 90, 90, 90, 469: 90, 90, 477: 90, 90, 90, 493: 90, 646: 90, 768: 90, 774: 90, 780: 90, 90}, {817, 817, 50: 817, 461: 817, 463: 817, 469: 817, 817, 477: 817, 817}, - {79, 79, 50: 79, 461: 79, 79, 79, 469: 79, 79, 477: 79, 79, 79, 501: 79, 770: 79, 1280: 3863, 3864}, + {79, 79, 50: 79, 461: 79, 79, 79, 469: 79, 79, 477: 79, 79, 79, 493: 79, 768: 79, 1278: 3864, 3865}, // 1405 - {77, 77, 50: 77, 461: 77, 77, 77, 469: 77, 77, 477: 77, 77, 77, 501: 77, 770: 3868, 1220: 3867}, - {649: 3865}, - {464: 3850, 565: 3851, 569: 3852, 840: 3866}, - {78, 78, 50: 78, 461: 78, 78, 78, 469: 78, 78, 477: 78, 78, 78, 501: 78, 770: 78}, - {80, 80, 50: 80, 461: 80, 80, 80, 469: 80, 80, 477: 80, 80, 80, 501: 80}, + {77, 77, 50: 77, 461: 77, 77, 77, 469: 77, 77, 477: 77, 77, 77, 493: 77, 768: 3869, 1219: 3868}, + {647: 3866}, + {464: 3851, 565: 3852, 569: 3853, 838: 3867}, + {78, 78, 50: 78, 461: 78, 78, 78, 469: 78, 78, 477: 78, 78, 78, 493: 78, 768: 78}, + {80, 80, 50: 80, 461: 80, 80, 80, 469: 80, 80, 477: 80, 80, 80, 493: 80}, // 1410 - {649: 3869}, - {464: 3850, 565: 3851, 569: 3852, 840: 3870}, - {76, 76, 50: 76, 461: 76, 76, 76, 469: 76, 76, 477: 76, 76, 76, 501: 76}, - {813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 469: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 482: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 497: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 533: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 559: 813, 571: 813, 636: 813, 644: 813, 732: 813}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3873}, + {647: 3870}, + {464: 3851, 565: 3852, 569: 3853, 838: 3871}, + {76, 76, 50: 76, 461: 76, 76, 76, 469: 76, 76, 477: 76, 76, 76, 493: 76}, + {813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 469: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 482: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 498: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 535: 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 813, 559: 813, 571: 813, 634: 813, 642: 813, 730: 813}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3874}, // 1415 - {7: 1879, 50: 3874, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 463: 1227, 1227, 1227, 1227, 1227, 469: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 482: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 492: 1227, 1227, 497: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 533: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 571: 1227}, - {1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 463: 1228, 1228, 1228, 1228, 1228, 469: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 482: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 492: 1228, 1228, 497: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 533: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 571: 1228}, - {1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 463: 1230, 1230, 1230, 1230, 1230, 469: 1230, 1230, 1230, 1230, 3251, 1230, 1230, 1230, 1230, 1230, 1230, 482: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 492: 1230, 1230, 497: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 533: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 571: 1230}, - {1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 463: 1232, 1232, 1232, 1232, 1232, 469: 1232, 1232, 1232, 1232, 3251, 1232, 1232, 1232, 1232, 1232, 1232, 482: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 492: 1232, 1232, 497: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 533: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 571: 1232}, + {7: 1879, 50: 3875, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 463: 1227, 1227, 1227, 1227, 1227, 469: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 482: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 492: 1227, 1227, 1227, 498: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 535: 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 1227, 571: 1227}, + {1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 463: 1228, 1228, 1228, 1228, 1228, 469: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 482: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 492: 1228, 1228, 1228, 498: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 535: 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 1228, 571: 1228}, + {1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 463: 1230, 1230, 1230, 1230, 1230, 469: 1230, 1230, 1230, 1230, 3250, 1230, 1230, 1230, 1230, 1230, 1230, 482: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 492: 1230, 1230, 1230, 498: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 535: 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 1230, 571: 1230}, + {1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 463: 1232, 1232, 1232, 1232, 1232, 469: 1232, 1232, 1232, 1232, 3250, 1232, 1232, 1232, 1232, 1232, 1232, 482: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 492: 1232, 1232, 1232, 498: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 535: 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 1232, 571: 1232}, // 1420 - {1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 463: 1233, 1233, 1233, 1233, 1233, 469: 1233, 1233, 1233, 1233, 3251, 1233, 1233, 1233, 1233, 1233, 1233, 482: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 492: 1233, 1233, 497: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 533: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 571: 1233}, - {1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 463: 1234, 1234, 1234, 1234, 1234, 469: 1234, 1234, 1234, 1234, 3251, 1234, 1234, 1234, 1234, 1234, 1234, 482: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 492: 1234, 1234, 497: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 533: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 571: 1234}, - {1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 463: 1235, 1235, 1235, 1235, 1235, 469: 1235, 1235, 1235, 1235, 3251, 1235, 1235, 1235, 1235, 1235, 1235, 482: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 492: 1235, 1235, 497: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 533: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 571: 1235}, + {1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 463: 1233, 1233, 1233, 1233, 1233, 469: 1233, 1233, 1233, 1233, 3250, 1233, 1233, 1233, 1233, 1233, 1233, 482: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 492: 1233, 1233, 1233, 498: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 535: 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 1233, 571: 1233}, + {1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 463: 1234, 1234, 1234, 1234, 1234, 469: 1234, 1234, 1234, 1234, 3250, 1234, 1234, 1234, 1234, 1234, 1234, 482: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 492: 1234, 1234, 1234, 498: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 535: 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 1234, 571: 1234}, + {1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 463: 1235, 1235, 1235, 1235, 1235, 469: 1235, 1235, 1235, 1235, 3250, 1235, 1235, 1235, 1235, 1235, 1235, 482: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 492: 1235, 1235, 1235, 498: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 535: 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 1235, 571: 1235}, + {464: 3885}, {464: 3884}, - {464: 3883}, // 1425 - {1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 463: 1215, 1215, 1215, 1215, 1215, 469: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 482: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 492: 1215, 1215, 497: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 533: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 571: 1215}, - {1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 463: 1216, 1216, 1216, 1216, 1216, 469: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 482: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 492: 1216, 1216, 497: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 533: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 571: 1216}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3886, 2674, 2675, 2673}, - {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3887, 1247, 1247, 1247, 1247, 1247, 469: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 482: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 492: 1247, 1247, 497: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 533: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 571: 1247, 642: 3682, 645: 1247, 1247}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3888}, + {1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 463: 1215, 1215, 1215, 1215, 1215, 469: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 482: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 492: 1215, 1215, 1215, 498: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 535: 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 1215, 571: 1215}, + {1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 463: 1216, 1216, 1216, 1216, 1216, 469: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 482: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 492: 1216, 1216, 1216, 498: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 535: 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 1216, 571: 1216}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3887, 2675, 2676, 2674}, + {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3888, 1247, 1247, 1247, 1247, 1247, 469: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 482: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 492: 1247, 1247, 1247, 498: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 535: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 571: 1247, 640: 3683, 643: 1247, 1247}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3889}, // 1430 - {50: 3889}, - {1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 463: 1083, 1083, 1083, 1083, 1083, 469: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 482: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 492: 1083, 1083, 497: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 533: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 571: 1083}, - {1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 463: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 492: 1285, 1285, 497: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 533: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 566: 1285, 571: 1285, 574: 1285, 576: 1285, 632: 1285, 1285, 1285, 1285}, - {1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 463: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 492: 1282, 1282, 497: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 533: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 566: 1282, 571: 1282, 574: 1282, 576: 1282, 632: 1282, 1282, 1282, 1282}, - {1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 463: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 492: 1281, 1281, 497: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 533: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 566: 1281, 571: 1281, 574: 1281, 576: 1281, 632: 1281, 1281, 1281, 1281}, + {50: 3890}, + {1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 463: 1083, 1083, 1083, 1083, 1083, 469: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 482: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 492: 1083, 1083, 1083, 498: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 535: 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 1083, 571: 1083}, + {1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 463: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 492: 1285, 1285, 1285, 498: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 535: 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 1285, 566: 1285, 571: 1285, 574: 1285, 576: 1285, 630: 1285, 1285, 1285, 1285}, + {1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 463: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 492: 1282, 1282, 1282, 498: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 535: 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 1282, 566: 1282, 571: 1282, 574: 1282, 576: 1282, 630: 1282, 1282, 1282, 1282}, + {1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 463: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 492: 1281, 1281, 1281, 498: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 535: 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 1281, 566: 1281, 571: 1281, 574: 1281, 576: 1281, 630: 1281, 1281, 1281, 1281}, // 1435 - {1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 463: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 492: 1279, 1279, 497: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 533: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 566: 1279, 571: 1279, 574: 1279, 576: 1279, 632: 1279, 1279, 1279, 1279}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3896, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3895}, - {50: 3900, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3897}, - {50: 3898, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 463: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 492: 1279, 1279, 1279, 498: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 535: 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 1279, 566: 1279, 571: 1279, 574: 1279, 576: 1279, 630: 1279, 1279, 1279, 1279}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3897, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3896}, + {50: 3901, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3898}, + {50: 3899, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1440 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3899}, - {1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 463: 1091, 1091, 1091, 1091, 1091, 469: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 482: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 492: 1091, 1091, 497: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 533: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 571: 1091}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3901}, - {1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 463: 1092, 1092, 1092, 1092, 1092, 469: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 482: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 492: 1092, 1092, 497: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 533: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 571: 1092}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3904, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3903}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3900}, + {1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 463: 1091, 1091, 1091, 1091, 1091, 469: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 482: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 492: 1091, 1091, 1091, 498: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 535: 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 1091, 571: 1091}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3902}, + {1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 463: 1092, 1092, 1092, 1092, 1092, 469: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 482: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 492: 1092, 1092, 1092, 498: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 535: 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 1092, 571: 1092}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3905, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3904}, // 1445 - {7: 3914, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3905}, - {7: 3906, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3908, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3907}, - {50: 3912, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {7: 3915, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3906}, + {7: 3907, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3909, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3908}, + {50: 3913, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1450 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3909}, - {50: 3910, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3911}, - {1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 463: 1087, 1087, 1087, 1087, 1087, 469: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 482: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 492: 1087, 1087, 497: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 533: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 571: 1087}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3913}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3910}, + {50: 3911, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3912}, + {1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 463: 1087, 1087, 1087, 1087, 1087, 469: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 482: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 492: 1087, 1087, 1087, 498: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 535: 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 1087, 571: 1087}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3914}, // 1455 - {1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 463: 1089, 1089, 1089, 1089, 1089, 469: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 482: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 492: 1089, 1089, 497: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 533: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 571: 1089}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 641: 3916, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3915}, - {50: 3920, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3917}, - {50: 3918, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 463: 1089, 1089, 1089, 1089, 1089, 469: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 482: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 492: 1089, 1089, 1089, 498: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 535: 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 1089, 571: 1089}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 639: 3917, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3916}, + {50: 3921, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3918}, + {50: 3919, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1460 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3919}, - {1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 463: 1088, 1088, 1088, 1088, 1088, 469: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 482: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 492: 1088, 1088, 497: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 533: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 571: 1088}, - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3921}, - {1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 463: 1090, 1090, 1090, 1090, 1090, 469: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 482: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 492: 1090, 1090, 497: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 533: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 571: 1090}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 794: 3923}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3920}, + {1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 463: 1088, 1088, 1088, 1088, 1088, 469: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 482: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 492: 1088, 1088, 1088, 498: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 535: 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 1088, 571: 1088}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3922}, + {1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 463: 1090, 1090, 1090, 1090, 1090, 469: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 482: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 492: 1090, 1090, 1090, 498: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 535: 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 1090, 571: 1090}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 792: 3924}, // 1465 - {7: 3924}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3925}, - {7: 3926, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3927}, - {50: 3928, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {7: 3925}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3926}, + {7: 3927, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3928}, + {50: 3929, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1470 - {1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 463: 1137, 1137, 1137, 1137, 1137, 469: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 482: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 492: 1137, 1137, 497: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 533: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 571: 1137}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 794: 3930}, - {7: 3931}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3932}, - {7: 3933, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 463: 1137, 1137, 1137, 1137, 1137, 469: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 482: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 492: 1137, 1137, 1137, 498: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 535: 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 1137, 571: 1137}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 792: 3931}, + {7: 3932}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3933}, + {7: 3934, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1475 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3934}, - {50: 3935, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 463: 1138, 1138, 1138, 1138, 1138, 469: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 482: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 492: 1138, 1138, 497: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 533: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 571: 1138}, - {171: 3939, 3938, 189: 3940, 214: 3941, 1199: 3937}, - {7: 3942}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3935}, + {50: 3936, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 463: 1138, 1138, 1138, 1138, 1138, 469: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 482: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 492: 1138, 1138, 1138, 498: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 535: 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 1138, 571: 1138}, + {171: 3940, 3939, 190: 3941, 214: 3942, 1198: 3938}, + {7: 3943}, // 1480 {7: 1127}, {7: 1126}, {7: 1125}, {7: 1124}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3943}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3944}, // 1485 - {50: 3944, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 463: 1144, 1144, 1144, 1144, 1144, 469: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 482: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 492: 1144, 1144, 497: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 533: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 571: 1144}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3946}, - {7: 3947}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 3949}, + {50: 3945, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 463: 1144, 1144, 1144, 1144, 1144, 469: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 482: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 492: 1144, 1144, 1144, 498: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 535: 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 1144, 571: 1144}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3947}, + {7: 3948}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 3950}, // 1490 - {1931, 1931, 4: 1931, 1931, 1931, 1931, 13: 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 50: 1931, 70: 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 106: 1931, 126: 1931, 1931, 1931, 1931, 468: 1931, 1931, 473: 1931, 483: 1931, 488: 1931, 1931, 492: 1931, 494: 1931, 637: 1931, 639: 1931, 647: 1931}, - {50: 3955}, - {29, 29, 4: 29, 29, 29, 13: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 50: 29, 70: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 468: 29, 29, 473: 29, 492: 29, 494: 29, 637: 29, 639: 29, 647: 29}, - {496: 2648, 725: 3948, 751: 3954}, - {496: 2648, 725: 3953}, + {1931, 1931, 4: 1931, 1931, 1931, 1931, 13: 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 50: 1931, 70: 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 1931, 106: 1931, 126: 1931, 1931, 1931, 1931, 468: 1931, 1931, 473: 1931, 483: 1931, 488: 1931, 1931, 492: 1931, 495: 1931, 635: 1931, 637: 1931, 645: 1931}, + {50: 3956}, + {29, 29, 4: 29, 29, 29, 13: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 50: 29, 70: 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 468: 29, 29, 473: 29, 492: 29, 495: 29, 635: 29, 637: 29, 645: 29}, + {497: 2649, 721: 3949, 749: 3955}, + {497: 2649, 721: 3954}, // 1495 - {27, 27, 4: 27, 27, 27, 13: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 50: 27, 70: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 468: 27, 27, 473: 27, 492: 27, 494: 27, 637: 27, 639: 27, 647: 27}, - {28, 28, 4: 28, 28, 28, 13: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 50: 28, 70: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 468: 28, 28, 473: 28, 492: 28, 494: 28, 637: 28, 639: 28, 647: 28}, - {1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 463: 1115, 1115, 1115, 1115, 1115, 469: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 482: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 492: 1115, 1115, 497: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 533: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 571: 1115}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3957}, - {50: 3958}, + {27, 27, 4: 27, 27, 27, 13: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 50: 27, 70: 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 468: 27, 27, 473: 27, 492: 27, 495: 27, 635: 27, 637: 27, 645: 27}, + {28, 28, 4: 28, 28, 28, 13: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 50: 28, 70: 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 468: 28, 28, 473: 28, 492: 28, 495: 28, 635: 28, 637: 28, 645: 28}, + {1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 463: 1115, 1115, 1115, 1115, 1115, 469: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 482: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 492: 1115, 1115, 1115, 498: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 535: 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 1115, 571: 1115}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3958}, + {50: 3959}, // 1500 - {1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 463: 1116, 1116, 1116, 1116, 1116, 469: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 482: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 492: 1116, 1116, 497: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 533: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 571: 1116}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3960}, - {50: 3961, 467: 3962, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 463: 1132, 1132, 1132, 1132, 1132, 469: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 482: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 492: 1132, 1132, 497: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 533: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 571: 1132}, - {494: 3708, 532: 3964, 637: 3707, 888: 3963}, + {1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 463: 1116, 1116, 1116, 1116, 1116, 469: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 482: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 492: 1116, 1116, 1116, 498: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 535: 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 1116, 571: 1116}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3961}, + {50: 3962, 467: 3963, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 463: 1132, 1132, 1132, 1132, 1132, 469: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 482: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 492: 1132, 1132, 1132, 498: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 535: 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 1132, 571: 1132}, + {495: 3709, 534: 3965, 635: 3708, 886: 3964}, // 1505 - {462: 3723, 750: 3967}, - {462: 3723, 750: 3965}, - {50: 3966}, - {1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 463: 1130, 1130, 1130, 1130, 1130, 469: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 482: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 492: 1130, 1130, 497: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 533: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 571: 1130}, - {50: 3968}, + {462: 3724, 748: 3968}, + {462: 3724, 748: 3966}, + {50: 3967}, + {1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 463: 1130, 1130, 1130, 1130, 1130, 469: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 482: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 492: 1130, 1130, 1130, 498: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 535: 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 1130, 571: 1130}, + {50: 3969}, // 1510 - {1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 463: 1131, 1131, 1131, 1131, 1131, 469: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 482: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 492: 1131, 1131, 497: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 533: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 571: 1131}, - {1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 463: 1154, 1154, 1154, 1154, 1154, 469: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 482: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 492: 1154, 1154, 497: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 533: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 571: 1154}, - {1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 463: 1155, 1155, 1155, 1155, 1155, 469: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 482: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 492: 1155, 1155, 497: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 533: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 571: 1155}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3972}, - {50: 3973}, + {1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 463: 1131, 1131, 1131, 1131, 1131, 469: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 482: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 492: 1131, 1131, 1131, 498: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 535: 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 1131, 571: 1131}, + {1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 463: 1154, 1154, 1154, 1154, 1154, 469: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 482: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 492: 1154, 1154, 1154, 498: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 535: 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 1154, 571: 1154}, + {1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 463: 1155, 1155, 1155, 1155, 1155, 469: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 482: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 492: 1155, 1155, 1155, 498: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 535: 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 1155, 571: 1155}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3973}, + {50: 3974}, // 1515 - {1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 463: 1151, 1151, 1151, 1151, 1151, 469: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 482: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 492: 1151, 1151, 497: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 533: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 571: 1151}, - {1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 463: 1156, 1156, 1156, 1156, 1156, 469: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 482: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 492: 1156, 1156, 497: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 533: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 571: 1156}, - {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 494: 1210, 1210, 1210, 532: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 638: 1210, 641: 3445, 735: 3443, 3444, 772: 3446, 775: 3447, 802: 3976, 804: 3448}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3977}, - {50: 3978, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 463: 1151, 1151, 1151, 1151, 1151, 469: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 482: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 492: 1151, 1151, 1151, 498: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 535: 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 1151, 571: 1151}, + {1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 463: 1156, 1156, 1156, 1156, 1156, 469: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 482: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 492: 1156, 1156, 1156, 498: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 535: 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 1156, 571: 1156}, + {2: 1210, 1210, 1210, 1210, 1210, 8: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 51: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 462: 1210, 464: 1210, 1210, 1210, 468: 1210, 471: 1210, 1210, 474: 1210, 1210, 1210, 481: 1210, 491: 1210, 495: 1210, 1210, 1210, 534: 1210, 555: 1210, 1210, 1210, 1210, 560: 1210, 1210, 1210, 1210, 1210, 1210, 567: 1210, 1210, 1210, 1210, 572: 1210, 1210, 575: 1210, 577: 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 1210, 636: 1210, 639: 3446, 733: 3444, 3445, 770: 3447, 773: 3448, 800: 3977, 802: 3449}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3978}, + {50: 3979, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 1520 - {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 497: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 533: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 730: 3263, 737: 3453, 755: 3979}, - {1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 463: 1113, 1113, 1113, 1113, 1113, 469: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 482: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 492: 1113, 1113, 497: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 533: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 571: 1113}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 3981}, - {50: 3982}, - {1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 463: 1084, 1084, 1084, 1084, 1084, 469: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 482: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 492: 1084, 1084, 497: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 533: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 571: 1084}, + {931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 463: 931, 931, 931, 931, 931, 469: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 482: 931, 931, 931, 931, 931, 931, 931, 931, 931, 492: 931, 931, 931, 498: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 535: 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 931, 571: 931, 728: 3262, 735: 3454, 753: 3980}, + {1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 463: 1113, 1113, 1113, 1113, 1113, 469: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 482: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 492: 1113, 1113, 1113, 498: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 535: 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 1113, 571: 1113}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 3982}, + {50: 3983}, + {1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 463: 1084, 1084, 1084, 1084, 1084, 469: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 482: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 492: 1084, 1084, 1084, 498: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 535: 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 1084, 571: 1084}, // 1525 - {147: 2244, 168: 2244, 184: 2244, 481: 2244, 510: 2244, 533: 2244, 544: 2244, 553: 2244, 2244, 560: 2244, 2244, 573: 2244}, - {147: 2243, 168: 2243, 184: 2243, 481: 2243, 510: 2243, 533: 2243, 544: 2243, 553: 2243, 2243, 560: 2243, 2243, 573: 2243}, - {2: 1858, 1858, 1858, 1858, 1858, 8: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 51: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 462: 1858, 464: 1858, 1858, 468: 1858, 471: 1858, 1858, 474: 1858, 1858, 1858, 481: 1858, 491: 1858, 494: 1858, 1858, 1858, 532: 1858, 555: 1858, 1858, 1858, 1858, 560: 1858, 1858, 1858, 1858, 1858, 1858, 567: 1858, 1858, 1858, 1858, 572: 1858, 1858, 575: 1858, 577: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858}, - {510: 4010, 533: 4009, 544: 4008, 553: 3994, 3995, 1092: 4011}, + {147: 2245, 168: 2245, 185: 2245, 481: 2245, 510: 2245, 532: 2245, 544: 2245, 553: 2245, 2245, 560: 2245, 2245, 573: 2245}, + {147: 2244, 168: 2244, 185: 2244, 481: 2244, 510: 2244, 532: 2244, 544: 2244, 553: 2244, 2244, 560: 2244, 2244, 573: 2244}, + {2: 1858, 1858, 1858, 1858, 1858, 8: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 51: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 462: 1858, 464: 1858, 1858, 468: 1858, 471: 1858, 1858, 474: 1858, 1858, 1858, 481: 1858, 491: 1858, 495: 1858, 1858, 1858, 534: 1858, 555: 1858, 1858, 1858, 1858, 560: 1858, 1858, 1858, 1858, 1858, 1858, 567: 1858, 1858, 1858, 1858, 572: 1858, 1858, 575: 1858, 577: 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858, 1858}, + {510: 4011, 532: 4010, 544: 4009, 553: 3995, 3996, 1091: 4012}, {462: 1854}, // 1530 - {2: 1852, 1852, 1852, 1852, 1852, 8: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 51: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 462: 1852, 464: 1852, 1852, 468: 1852, 471: 1852, 1852, 474: 1852, 1852, 1852, 481: 1852, 491: 1852, 494: 1852, 1852, 1852, 532: 1852, 555: 1852, 1852, 1852, 1852, 560: 1852, 1852, 1852, 1852, 1852, 1852, 567: 1852, 1852, 1852, 1852, 572: 1852, 1852, 575: 1852, 577: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852}, - {2: 1850, 1850, 1850, 1850, 1850, 8: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 51: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 462: 1850, 464: 1850, 1850, 468: 1850, 471: 1850, 1850, 474: 1850, 1850, 1850, 481: 1850, 491: 1850, 494: 1850, 1850, 1850, 532: 1850, 555: 1850, 1850, 1850, 1850, 560: 1850, 1850, 1850, 1850, 1850, 1850, 567: 1850, 1850, 1850, 1850, 572: 1850, 1850, 575: 1850, 577: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850}, - {462: 4004, 699: 4005}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4001}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3997, 3153, 3236, 3152, 3149}, + {2: 1852, 1852, 1852, 1852, 1852, 8: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 51: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 462: 1852, 464: 1852, 1852, 468: 1852, 471: 1852, 1852, 474: 1852, 1852, 1852, 481: 1852, 491: 1852, 495: 1852, 1852, 1852, 534: 1852, 555: 1852, 1852, 1852, 1852, 560: 1852, 1852, 1852, 1852, 1852, 1852, 567: 1852, 1852, 1852, 1852, 572: 1852, 1852, 575: 1852, 577: 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852, 1852}, + {2: 1850, 1850, 1850, 1850, 1850, 8: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 51: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 462: 1850, 464: 1850, 1850, 468: 1850, 471: 1850, 1850, 474: 1850, 1850, 1850, 481: 1850, 491: 1850, 495: 1850, 1850, 1850, 534: 1850, 555: 1850, 1850, 1850, 1850, 560: 1850, 1850, 1850, 1850, 1850, 1850, 567: 1850, 1850, 1850, 1850, 572: 1850, 1850, 575: 1850, 577: 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850, 1850}, + {462: 4005, 697: 4006}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4002}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3998, 3154, 3235, 3153, 3150}, // 1535 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3996, 3153, 3236, 3152, 3149}, - {2: 1839, 1839, 1839, 1839, 1839, 8: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 51: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 462: 1839, 464: 1839, 1839, 468: 1839, 471: 1839, 1839, 474: 1839, 1839, 1839, 481: 1839, 491: 1839, 494: 1839, 1839, 1839, 532: 1839, 555: 1839, 1839, 1839, 1839, 560: 1839, 1839, 1839, 1839, 1839, 1839, 567: 1839, 1839, 1839, 1839, 572: 1839, 1839, 575: 1839, 577: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839}, - {2: 1838, 1838, 1838, 1838, 1838, 8: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 51: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 462: 1838, 464: 1838, 1838, 468: 1838, 471: 1838, 1838, 474: 1838, 1838, 1838, 481: 1838, 491: 1838, 494: 1838, 1838, 1838, 532: 1838, 555: 1838, 1838, 1838, 1838, 560: 1838, 1838, 1838, 1838, 1838, 1838, 567: 1838, 1838, 1838, 1838, 572: 1838, 1838, 575: 1838, 577: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838}, - {1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 463: 1841, 1841, 467: 1841, 469: 1841, 1841, 1841, 1841, 3251, 477: 1841, 1841, 1841, 482: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 492: 1841, 1841, 497: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 511: 1841, 513: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 534: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 571: 3252}, - {1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 3999, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 463: 1837, 1837, 467: 1837, 469: 1837, 1837, 1837, 1837, 3251, 477: 1837, 1837, 1837, 482: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 492: 1837, 1837, 497: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 511: 1837, 513: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 534: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 571: 3252, 1217: 3998}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3997, 3154, 3235, 3153, 3150}, + {2: 1839, 1839, 1839, 1839, 1839, 8: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 51: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 462: 1839, 464: 1839, 1839, 468: 1839, 471: 1839, 1839, 474: 1839, 1839, 1839, 481: 1839, 491: 1839, 495: 1839, 1839, 1839, 534: 1839, 555: 1839, 1839, 1839, 1839, 560: 1839, 1839, 1839, 1839, 1839, 1839, 567: 1839, 1839, 1839, 1839, 572: 1839, 1839, 575: 1839, 577: 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839, 1839}, + {2: 1838, 1838, 1838, 1838, 1838, 8: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 51: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 462: 1838, 464: 1838, 1838, 468: 1838, 471: 1838, 1838, 474: 1838, 1838, 1838, 481: 1838, 491: 1838, 495: 1838, 1838, 1838, 534: 1838, 555: 1838, 1838, 1838, 1838, 560: 1838, 1838, 1838, 1838, 1838, 1838, 567: 1838, 1838, 1838, 1838, 572: 1838, 1838, 575: 1838, 577: 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838, 1838}, + {1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 463: 1841, 1841, 467: 1841, 469: 1841, 1841, 1841, 1841, 3250, 477: 1841, 1841, 1841, 482: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 492: 1841, 1841, 1841, 498: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 511: 1841, 513: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 533: 1841, 535: 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 1841, 571: 3251}, + {1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 4000, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 463: 1837, 1837, 467: 1837, 469: 1837, 1837, 1837, 1837, 3250, 477: 1837, 1837, 1837, 482: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 492: 1837, 1837, 1837, 498: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 511: 1837, 513: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 533: 1837, 535: 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 1837, 571: 3251, 1216: 3999}, // 1540 - {1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 463: 1842, 1842, 467: 1842, 469: 1842, 1842, 1842, 1842, 477: 1842, 1842, 1842, 482: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 492: 1842, 1842, 497: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 511: 1842, 513: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 534: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842}, - {464: 4000}, - {1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 463: 1836, 1836, 467: 1836, 469: 1836, 1836, 1836, 1836, 477: 1836, 1836, 1836, 482: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 492: 1836, 1836, 497: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 511: 1836, 513: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 534: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836}, - {474: 3586, 3585, 3591, 493: 4002, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 4003}, + {1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 463: 1842, 1842, 467: 1842, 469: 1842, 1842, 1842, 1842, 477: 1842, 1842, 1842, 482: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 492: 1842, 1842, 1842, 498: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 511: 1842, 513: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 533: 1842, 535: 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842, 1842}, + {464: 4001}, + {1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 463: 1836, 1836, 467: 1836, 469: 1836, 1836, 1836, 1836, 477: 1836, 1836, 1836, 482: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 492: 1836, 1836, 1836, 498: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 511: 1836, 513: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 533: 1836, 535: 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836, 1836}, + {474: 3587, 3586, 3592, 494: 4003, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 4004}, // 1545 - {1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 463: 1843, 1843, 467: 1843, 469: 1843, 1843, 1843, 1843, 477: 1843, 1843, 1843, 482: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 492: 1843, 1843, 497: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 511: 1843, 513: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 534: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 2506, 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3776, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 2504, 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 636: 2500, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3775, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 741: 3778, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 3780, 3779, 3777, 765: 4006}, - {1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 463: 1844, 1844, 467: 1844, 469: 1844, 1844, 1844, 1844, 477: 1844, 1844, 1844, 482: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 492: 1844, 1844, 497: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 511: 1844, 513: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 534: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844}, - {7: 3489, 50: 4007}, - {1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 463: 1845, 1845, 467: 1845, 469: 1845, 1845, 1845, 1845, 477: 1845, 1845, 1845, 482: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 492: 1845, 1845, 497: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 511: 1845, 513: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 534: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845}, + {1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 463: 1843, 1843, 467: 1843, 469: 1843, 1843, 1843, 1843, 477: 1843, 1843, 1843, 482: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 492: 1843, 1843, 1843, 498: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 511: 1843, 513: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 533: 1843, 535: 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843, 1843}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 2507, 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3777, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 2505, 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 634: 2501, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3776, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 739: 3779, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 3781, 3780, 3778, 763: 4007}, + {1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 463: 1844, 1844, 467: 1844, 469: 1844, 1844, 1844, 1844, 477: 1844, 1844, 1844, 482: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 492: 1844, 1844, 1844, 498: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 511: 1844, 513: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 533: 1844, 535: 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844, 1844}, + {7: 3490, 50: 4008}, + {1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 463: 1845, 1845, 467: 1845, 469: 1845, 1845, 1845, 1845, 477: 1845, 1845, 1845, 482: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 492: 1845, 1845, 1845, 498: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 511: 1845, 513: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 533: 1845, 535: 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845, 1845}, // 1550 - {2: 1857, 1857, 1857, 1857, 1857, 8: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 51: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 462: 1857, 464: 1857, 1857, 468: 1857, 471: 1857, 1857, 474: 1857, 1857, 1857, 481: 1857, 491: 1857, 494: 1857, 1857, 1857, 532: 1857, 555: 1857, 1857, 1857, 1857, 560: 1857, 1857, 1857, 1857, 1857, 1857, 567: 1857, 1857, 1857, 1857, 572: 1857, 1857, 575: 1857, 577: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857}, + {2: 1857, 1857, 1857, 1857, 1857, 8: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 51: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 462: 1857, 464: 1857, 1857, 468: 1857, 471: 1857, 1857, 474: 1857, 1857, 1857, 481: 1857, 491: 1857, 495: 1857, 1857, 1857, 534: 1857, 555: 1857, 1857, 1857, 1857, 560: 1857, 1857, 1857, 1857, 1857, 1857, 567: 1857, 1857, 1857, 1857, 572: 1857, 1857, 575: 1857, 577: 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857, 1857}, {462: 1853}, - {2: 1851, 1851, 1851, 1851, 1851, 8: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 51: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 462: 1851, 464: 1851, 1851, 468: 1851, 471: 1851, 1851, 474: 1851, 1851, 1851, 481: 1851, 491: 1851, 494: 1851, 1851, 1851, 532: 1851, 555: 1851, 1851, 1851, 1851, 560: 1851, 1851, 1851, 1851, 1851, 1851, 567: 1851, 1851, 1851, 1851, 572: 1851, 1851, 575: 1851, 577: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851}, - {2: 1849, 1849, 1849, 1849, 1849, 8: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 51: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 462: 1849, 464: 1849, 1849, 468: 1849, 471: 1849, 1849, 474: 1849, 1849, 1849, 481: 1849, 491: 1849, 494: 1849, 1849, 1849, 532: 1849, 555: 1849, 1849, 1849, 1849, 560: 1849, 1849, 1849, 1849, 1849, 1849, 567: 1849, 1849, 1849, 1849, 572: 1849, 1849, 575: 1849, 577: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849}, - {168: 4035, 481: 4036, 560: 4034, 4033}, + {2: 1851, 1851, 1851, 1851, 1851, 8: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 51: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 462: 1851, 464: 1851, 1851, 468: 1851, 471: 1851, 1851, 474: 1851, 1851, 1851, 481: 1851, 491: 1851, 495: 1851, 1851, 1851, 534: 1851, 555: 1851, 1851, 1851, 1851, 560: 1851, 1851, 1851, 1851, 1851, 1851, 567: 1851, 1851, 1851, 1851, 572: 1851, 1851, 575: 1851, 577: 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851, 1851}, + {2: 1849, 1849, 1849, 1849, 1849, 8: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 51: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 462: 1849, 464: 1849, 1849, 468: 1849, 471: 1849, 1849, 474: 1849, 1849, 1849, 481: 1849, 491: 1849, 495: 1849, 1849, 1849, 534: 1849, 555: 1849, 1849, 1849, 1849, 560: 1849, 1849, 1849, 1849, 1849, 1849, 567: 1849, 1849, 1849, 1849, 572: 1849, 1849, 575: 1849, 577: 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849, 1849}, + {168: 4036, 481: 4037, 560: 4035, 4034}, // 1555 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 4027, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 4028, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 4026, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 641: 4029, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 4024, 1152: 4025}, - {2: 1866, 1866, 1866, 1866, 1866, 8: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 51: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 462: 1866, 464: 1866, 1866, 468: 1866, 471: 1866, 1866, 474: 1866, 1866, 1866, 481: 1866, 491: 1866, 494: 1866, 1866, 1866, 532: 1866, 555: 1866, 1866, 1866, 1866, 560: 1866, 1866, 1866, 1866, 1866, 1866, 567: 1866, 1866, 1866, 1866, 572: 1866, 1866, 575: 1866, 577: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 641: 1866}, - {2: 1865, 1865, 1865, 1865, 1865, 8: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 51: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 462: 1865, 464: 1865, 1865, 468: 1865, 471: 1865, 1865, 474: 1865, 1865, 1865, 481: 1865, 491: 1865, 494: 1865, 1865, 1865, 532: 1865, 555: 1865, 1865, 1865, 1865, 560: 1865, 1865, 1865, 1865, 1865, 1865, 567: 1865, 1865, 1865, 1865, 572: 1865, 1865, 575: 1865, 577: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 641: 1865}, - {2: 1864, 1864, 1864, 1864, 1864, 8: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 51: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 462: 1864, 464: 1864, 1864, 468: 1864, 471: 1864, 1864, 474: 1864, 1864, 1864, 481: 1864, 491: 1864, 494: 1864, 1864, 1864, 532: 1864, 555: 1864, 1864, 1864, 1864, 560: 1864, 1864, 1864, 1864, 1864, 1864, 567: 1864, 1864, 1864, 1864, 572: 1864, 1864, 575: 1864, 577: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 641: 1864}, - {2: 1863, 1863, 1863, 1863, 1863, 8: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 51: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 462: 1863, 464: 1863, 1863, 468: 1863, 471: 1863, 1863, 474: 1863, 1863, 1863, 481: 1863, 491: 1863, 494: 1863, 1863, 1863, 532: 1863, 555: 1863, 1863, 1863, 1863, 560: 1863, 1863, 1863, 1863, 1863, 1863, 567: 1863, 1863, 1863, 1863, 572: 1863, 1863, 575: 1863, 577: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 641: 1863}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 4028, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 4029, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 4027, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 639: 4030, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 4025, 1151: 4026}, + {2: 1866, 1866, 1866, 1866, 1866, 8: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 51: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 462: 1866, 464: 1866, 1866, 468: 1866, 471: 1866, 1866, 474: 1866, 1866, 1866, 481: 1866, 491: 1866, 495: 1866, 1866, 1866, 534: 1866, 555: 1866, 1866, 1866, 1866, 560: 1866, 1866, 1866, 1866, 1866, 1866, 567: 1866, 1866, 1866, 1866, 572: 1866, 1866, 575: 1866, 577: 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 639: 1866}, + {2: 1865, 1865, 1865, 1865, 1865, 8: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 51: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 462: 1865, 464: 1865, 1865, 468: 1865, 471: 1865, 1865, 474: 1865, 1865, 1865, 481: 1865, 491: 1865, 495: 1865, 1865, 1865, 534: 1865, 555: 1865, 1865, 1865, 1865, 560: 1865, 1865, 1865, 1865, 1865, 1865, 567: 1865, 1865, 1865, 1865, 572: 1865, 1865, 575: 1865, 577: 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 1865, 639: 1865}, + {2: 1864, 1864, 1864, 1864, 1864, 8: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 51: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 462: 1864, 464: 1864, 1864, 468: 1864, 471: 1864, 1864, 474: 1864, 1864, 1864, 481: 1864, 491: 1864, 495: 1864, 1864, 1864, 534: 1864, 555: 1864, 1864, 1864, 1864, 560: 1864, 1864, 1864, 1864, 1864, 1864, 567: 1864, 1864, 1864, 1864, 572: 1864, 1864, 575: 1864, 577: 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 1864, 639: 1864}, + {2: 1863, 1863, 1863, 1863, 1863, 8: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 51: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 462: 1863, 464: 1863, 1863, 468: 1863, 471: 1863, 1863, 474: 1863, 1863, 1863, 481: 1863, 491: 1863, 495: 1863, 1863, 1863, 534: 1863, 555: 1863, 1863, 1863, 1863, 560: 1863, 1863, 1863, 1863, 1863, 1863, 567: 1863, 1863, 1863, 1863, 572: 1863, 1863, 575: 1863, 577: 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 1863, 639: 1863}, // 1560 - {2: 1862, 1862, 1862, 1862, 1862, 8: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 51: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 462: 1862, 464: 1862, 1862, 468: 1862, 471: 1862, 1862, 474: 1862, 1862, 1862, 481: 1862, 491: 1862, 494: 1862, 1862, 1862, 532: 1862, 555: 1862, 1862, 1862, 1862, 560: 1862, 1862, 1862, 1862, 1862, 1862, 567: 1862, 1862, 1862, 1862, 572: 1862, 1862, 575: 1862, 577: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 641: 1862}, - {2: 1861, 1861, 1861, 1861, 1861, 8: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 51: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 462: 1861, 464: 1861, 1861, 468: 1861, 471: 1861, 1861, 474: 1861, 1861, 1861, 481: 1861, 491: 1861, 494: 1861, 1861, 1861, 532: 1861, 555: 1861, 1861, 1861, 1861, 560: 1861, 1861, 1861, 1861, 1861, 1861, 567: 1861, 1861, 1861, 1861, 572: 1861, 1861, 575: 1861, 577: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 641: 1861}, - {2: 1860, 1860, 1860, 1860, 1860, 8: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 51: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 462: 1860, 464: 1860, 1860, 468: 1860, 471: 1860, 1860, 474: 1860, 1860, 1860, 481: 1860, 491: 1860, 494: 1860, 1860, 1860, 532: 1860, 555: 1860, 1860, 1860, 1860, 560: 1860, 1860, 1860, 1860, 1860, 1860, 567: 1860, 1860, 1860, 1860, 572: 1860, 1860, 575: 1860, 577: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 641: 1860}, - {2: 1859, 1859, 1859, 1859, 1859, 8: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 51: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 462: 1859, 464: 1859, 1859, 468: 1859, 471: 1859, 1859, 474: 1859, 1859, 1859, 481: 1859, 491: 1859, 494: 1859, 1859, 1859, 532: 1859, 555: 1859, 1859, 1859, 1859, 560: 1859, 1859, 1859, 1859, 1859, 1859, 567: 1859, 1859, 1859, 1859, 572: 1859, 1859, 575: 1859, 577: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 641: 1859}, - {168: 1856, 465: 3984, 3983, 481: 1856, 560: 1856, 1856, 798: 4023}, + {2: 1862, 1862, 1862, 1862, 1862, 8: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 51: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 462: 1862, 464: 1862, 1862, 468: 1862, 471: 1862, 1862, 474: 1862, 1862, 1862, 481: 1862, 491: 1862, 495: 1862, 1862, 1862, 534: 1862, 555: 1862, 1862, 1862, 1862, 560: 1862, 1862, 1862, 1862, 1862, 1862, 567: 1862, 1862, 1862, 1862, 572: 1862, 1862, 575: 1862, 577: 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 1862, 639: 1862}, + {2: 1861, 1861, 1861, 1861, 1861, 8: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 51: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 462: 1861, 464: 1861, 1861, 468: 1861, 471: 1861, 1861, 474: 1861, 1861, 1861, 481: 1861, 491: 1861, 495: 1861, 1861, 1861, 534: 1861, 555: 1861, 1861, 1861, 1861, 560: 1861, 1861, 1861, 1861, 1861, 1861, 567: 1861, 1861, 1861, 1861, 572: 1861, 1861, 575: 1861, 577: 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 1861, 639: 1861}, + {2: 1860, 1860, 1860, 1860, 1860, 8: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 51: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 462: 1860, 464: 1860, 1860, 468: 1860, 471: 1860, 1860, 474: 1860, 1860, 1860, 481: 1860, 491: 1860, 495: 1860, 1860, 1860, 534: 1860, 555: 1860, 1860, 1860, 1860, 560: 1860, 1860, 1860, 1860, 1860, 1860, 567: 1860, 1860, 1860, 1860, 572: 1860, 1860, 575: 1860, 577: 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 1860, 639: 1860}, + {2: 1859, 1859, 1859, 1859, 1859, 8: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 51: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 462: 1859, 464: 1859, 1859, 468: 1859, 471: 1859, 1859, 474: 1859, 1859, 1859, 481: 1859, 491: 1859, 495: 1859, 1859, 1859, 534: 1859, 555: 1859, 1859, 1859, 1859, 560: 1859, 1859, 1859, 1859, 1859, 1859, 567: 1859, 1859, 1859, 1859, 572: 1859, 1859, 575: 1859, 577: 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 1859, 639: 1859}, + {168: 1856, 465: 3985, 3984, 481: 1856, 560: 1856, 1856, 796: 4024}, // 1565 {168: 1855, 481: 1855, 560: 1855, 1855}, - {1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 463: 1870, 1870, 467: 1870, 469: 1870, 1870, 1870, 1870, 477: 1870, 1870, 1870, 482: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 492: 1870, 1870, 497: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 511: 1870, 513: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 534: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870}, - {462: 2507, 699: 4032}, - {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 497: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 533: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721, 650: 4030}, - {1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1848, 1672, 1672, 1672, 1672, 1672, 469: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 482: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 492: 1672, 1672, 497: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 533: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 571: 1672, 642: 1672, 645: 1672, 1672}, + {1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 463: 1870, 1870, 467: 1870, 469: 1870, 1870, 1870, 1870, 477: 1870, 1870, 1870, 482: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 492: 1870, 1870, 1870, 498: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 511: 1870, 513: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 533: 1870, 535: 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870, 1870}, + {462: 2508, 697: 4033}, + {721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 463: 721, 721, 721, 721, 721, 469: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 482: 721, 721, 721, 721, 721, 721, 721, 721, 721, 492: 721, 721, 721, 498: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 535: 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 721, 571: 721, 648: 4031}, + {1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1848, 1672, 1672, 1672, 1672, 1672, 469: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 482: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 492: 1672, 1672, 1672, 498: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 535: 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 1672, 571: 1672, 640: 1672, 643: 1672, 1672}, // 1570 - {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1847, 1671, 1671, 1671, 1671, 1671, 469: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 482: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 492: 1671, 1671, 497: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 533: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 571: 1671, 642: 1671, 645: 1671, 1671}, + {1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1847, 1671, 1671, 1671, 1671, 1671, 469: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 482: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 492: 1671, 1671, 1671, 498: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 535: 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 1671, 571: 1671, 640: 1671, 643: 1671, 1671}, {462: 1846}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 4031}, - {1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 463: 1868, 1868, 467: 1868, 469: 1868, 1868, 1868, 1868, 477: 1868, 1868, 1868, 482: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 492: 1868, 1868, 497: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 511: 1868, 513: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 534: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868}, - {1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 463: 1869, 1869, 467: 1869, 469: 1869, 1869, 1869, 1869, 477: 1869, 1869, 1869, 482: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 492: 1869, 1869, 497: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 511: 1869, 513: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 534: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 4032}, + {1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 463: 1868, 1868, 467: 1868, 469: 1868, 1868, 1868, 1868, 477: 1868, 1868, 1868, 482: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 492: 1868, 1868, 1868, 498: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 511: 1868, 513: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 533: 1868, 535: 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868, 1868}, + {1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 463: 1869, 1869, 467: 1869, 469: 1869, 1869, 1869, 1869, 477: 1869, 1869, 1869, 482: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 492: 1869, 1869, 1869, 498: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 511: 1869, 513: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 533: 1869, 535: 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869, 1869}, // 1575 - {1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 463: 1895, 1895, 467: 1895, 469: 1895, 1895, 1895, 1895, 477: 1895, 1895, 1895, 482: 1895, 1895, 1895, 1895, 1895, 488: 1895, 1895, 1895, 492: 1895, 1895, 497: 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 511: 1895, 513: 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 534: 1895, 1895}, - {1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 463: 1894, 1894, 467: 1894, 469: 1894, 1894, 1894, 1894, 477: 1894, 1894, 1894, 482: 1894, 1894, 1894, 1894, 1894, 488: 1894, 1894, 1894, 492: 1894, 1894, 497: 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 511: 1894, 513: 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 534: 1894, 1894}, - {1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 463: 1893, 1893, 467: 1893, 469: 1893, 1893, 1893, 1893, 477: 1893, 1893, 1893, 482: 1893, 1893, 1893, 1893, 1893, 488: 1893, 1893, 1893, 492: 1893, 1893, 497: 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 511: 1893, 513: 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 534: 1893, 1893}, - {1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 463: 1871, 1871, 467: 1871, 469: 1871, 1871, 1871, 1871, 477: 1871, 1871, 1871, 482: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 492: 1871, 1871, 497: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 511: 1871, 513: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 534: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 4040}, + {1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 463: 1895, 1895, 467: 1895, 469: 1895, 1895, 1895, 1895, 477: 1895, 1895, 1895, 482: 1895, 1895, 1895, 1895, 1895, 488: 1895, 1895, 1895, 492: 1895, 1895, 1895, 498: 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 511: 1895, 513: 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 1895, 533: 1895, 535: 1895}, + {1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 463: 1894, 1894, 467: 1894, 469: 1894, 1894, 1894, 1894, 477: 1894, 1894, 1894, 482: 1894, 1894, 1894, 1894, 1894, 488: 1894, 1894, 1894, 492: 1894, 1894, 1894, 498: 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 511: 1894, 513: 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 1894, 533: 1894, 535: 1894}, + {1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 463: 1893, 1893, 467: 1893, 469: 1893, 1893, 1893, 1893, 477: 1893, 1893, 1893, 482: 1893, 1893, 1893, 1893, 1893, 488: 1893, 1893, 1893, 492: 1893, 1893, 1893, 498: 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 511: 1893, 513: 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 1893, 533: 1893, 535: 1893}, + {1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 463: 1871, 1871, 467: 1871, 469: 1871, 1871, 1871, 1871, 477: 1871, 1871, 1871, 482: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 492: 1871, 1871, 1871, 498: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 511: 1871, 513: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 533: 1871, 535: 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871, 1871}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 4041}, // 1580 - {2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 462: 2267, 480: 2267, 487: 2267, 494: 2267, 501: 2267, 518: 2267, 2267, 532: 2267, 637: 2267, 642: 4061, 659: 2267, 2267, 662: 2267, 667: 2267, 2267, 670: 2267, 2267, 2267, 2267, 2267, 2267, 678: 2267, 680: 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 695: 2267, 2267, 2267, 2267}, - {7: 2264, 50: 2264}, - {7: 4041, 50: 4042}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4060}, - {293: 4043}, + {2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 462: 2268, 480: 2268, 487: 2268, 493: 2268, 495: 2268, 518: 2268, 2268, 534: 2268, 635: 2268, 640: 4062, 657: 2268, 2268, 660: 2268, 665: 2268, 2268, 668: 2268, 2268, 2268, 2268, 2268, 2268, 676: 2268, 678: 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 2268, 693: 2268, 2268, 2268, 2268}, + {7: 2265, 50: 2265}, + {7: 4042, 50: 4043}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4061}, + {295: 4044}, // 1585 - {462: 4044}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4045}, - {50: 1889, 463: 4048, 474: 3586, 3585, 3591, 512: 3587, 533: 4047, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584, 1197: 4046}, - {50: 4059}, - {220: 4052, 507: 4051}, + {462: 4045}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4046}, + {50: 1889, 463: 4049, 474: 3587, 3586, 3592, 512: 3588, 532: 4048, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585, 1196: 4047}, + {50: 4060}, + {220: 4053, 507: 4052}, // 1590 - {143: 4049}, - {242: 4050}, + {143: 4050}, + {242: 4051}, {50: 1885}, - {337: 4054}, - {203: 4053}, + {339: 4055}, + {203: 4054}, // 1595 {50: 1886}, - {203: 4055}, - {50: 1888, 463: 4056}, - {143: 4057}, - {242: 4058}, + {203: 4056}, + {50: 1888, 463: 4057}, + {143: 4058}, + {242: 4059}, // 1600 {50: 1887}, - {1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 463: 1896, 1896, 467: 1896, 469: 1896, 1896, 1896, 1896, 477: 1896, 1896, 1896, 482: 1896, 1896, 1896, 1896, 1896, 488: 1896, 1896, 1896, 492: 1896, 1896, 497: 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 511: 1896, 513: 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 534: 1896, 1896}, - {7: 2263, 50: 2263}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4062, 2674, 2675, 2673}, - {2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 462: 2266, 480: 2266, 487: 2266, 494: 2266, 501: 2266, 518: 2266, 2266, 532: 2266, 637: 2266, 642: 4063, 659: 2266, 2266, 662: 2266, 667: 2266, 2266, 670: 2266, 2266, 2266, 2266, 2266, 2266, 678: 2266, 680: 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 695: 2266, 2266, 2266, 2266}, + {1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 463: 1896, 1896, 467: 1896, 469: 1896, 1896, 1896, 1896, 477: 1896, 1896, 1896, 482: 1896, 1896, 1896, 1896, 1896, 488: 1896, 1896, 1896, 492: 1896, 1896, 1896, 498: 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 511: 1896, 513: 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 1896, 533: 1896, 535: 1896}, + {7: 2264, 50: 2264}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4063, 2675, 2676, 2674}, + {2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 462: 2267, 480: 2267, 487: 2267, 493: 2267, 495: 2267, 518: 2267, 2267, 534: 2267, 635: 2267, 640: 4064, 657: 2267, 2267, 660: 2267, 665: 2267, 2267, 668: 2267, 2267, 2267, 2267, 2267, 2267, 676: 2267, 678: 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 2267, 693: 2267, 2267, 2267, 2267}, // 1605 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4064, 2674, 2675, 2673}, - {2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 462: 2265, 480: 2265, 487: 2265, 494: 2265, 501: 2265, 518: 2265, 2265, 532: 2265, 637: 2265, 659: 2265, 2265, 662: 2265, 667: 2265, 2265, 670: 2265, 2265, 2265, 2265, 2265, 2265, 678: 2265, 680: 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 2265, 695: 2265, 2265, 2265, 2265}, - {1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 463: 1897, 1897, 467: 1897, 469: 1897, 1897, 1897, 1897, 477: 1897, 1897, 1897, 482: 1897, 1897, 1897, 1897, 1897, 488: 1897, 1897, 1897, 492: 1897, 1897, 497: 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 511: 1897, 513: 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 534: 1897, 1897, 723: 3243, 3241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4065, 2675, 2676, 2674}, + {2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 462: 2266, 480: 2266, 487: 2266, 493: 2266, 495: 2266, 518: 2266, 2266, 534: 2266, 635: 2266, 657: 2266, 2266, 660: 2266, 665: 2266, 2266, 668: 2266, 2266, 2266, 2266, 2266, 2266, 676: 2266, 678: 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 2266, 693: 2266, 2266, 2266, 2266}, + {1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 463: 1897, 1897, 467: 1897, 469: 1897, 1897, 1897, 1897, 477: 1897, 1897, 1897, 482: 1897, 1897, 1897, 1897, 1897, 488: 1897, 1897, 1897, 492: 1897, 1897, 1897, 498: 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 511: 1897, 513: 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 1897, 533: 1897, 535: 1897, 722: 3242, 3240}, {1271, 1271, 7: 1271, 50: 1271, 124: 1271, 461: 1271, 463: 1271, 469: 1271, 1271, 477: 1271, 1271, 482: 1271, 1271, 1271, 1271, 488: 1271, 490: 1271, 504: 1271, 1271, 513: 1271, 516: 1271, 1271}, {1270, 1270, 7: 1270, 50: 1270, 124: 1270, 461: 1270, 463: 1270, 469: 1270, 1270, 477: 1270, 1270, 482: 1270, 1270, 1270, 1270, 488: 1270, 490: 1270, 504: 1270, 1270, 513: 1270, 516: 1270, 1270}, // 1610 {1269, 1269, 7: 1269, 50: 1269, 124: 1269, 461: 1269, 463: 1269, 469: 1269, 1269, 477: 1269, 1269, 482: 1269, 1269, 1269, 1269, 488: 1269, 490: 1269, 504: 1269, 1269, 513: 1269, 516: 1269, 1269}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4070}, - {1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 463: 1901, 1901, 467: 1901, 469: 1901, 1901, 1901, 1901, 477: 1901, 1901, 1901, 482: 1901, 1901, 1901, 1901, 1901, 488: 1901, 1901, 1901, 492: 1901, 3247, 497: 3245, 3246, 3244, 3242, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 511: 1901, 513: 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 534: 1901, 1901, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4072}, - {50: 4073}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4071}, + {1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 463: 1901, 1901, 467: 1901, 469: 1901, 1901, 1901, 1901, 477: 1901, 1901, 1901, 482: 1901, 1901, 1901, 1901, 1901, 488: 1901, 1901, 1901, 492: 1901, 1901, 3246, 498: 3244, 3245, 3243, 3241, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 511: 1901, 513: 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 1901, 533: 1901, 535: 1901, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4073}, + {50: 4074}, // 1615 - {2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 463: 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 492: 2172, 2172, 497: 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 533: 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 2172, 566: 2172, 571: 2172, 574: 2172, 576: 2172, 632: 2172, 2172, 2172, 2172}, - {482: 4075}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4076}, - {2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 463: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 492: 2173, 2173, 497: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 533: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 566: 2173, 571: 2173, 574: 2173, 576: 2173, 632: 2173, 2173, 2173, 2173}, - {238, 238, 50: 238, 461: 238, 463: 238, 469: 238, 238, 477: 238, 238, 482: 238, 238, 238, 238, 488: 238, 490: 238, 493: 3247, 497: 3245, 3246, 3244, 3242, 502: 238, 504: 238, 238, 723: 3243, 3241}, + {2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 463: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 492: 2173, 2173, 2173, 498: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 535: 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 2173, 566: 2173, 571: 2173, 574: 2173, 576: 2173, 630: 2173, 2173, 2173, 2173}, + {482: 4076}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4077}, + {2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 463: 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 492: 2174, 2174, 2174, 498: 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 535: 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 566: 2174, 571: 2174, 574: 2174, 576: 2174, 630: 2174, 2174, 2174, 2174}, + {238, 238, 50: 238, 461: 238, 463: 238, 469: 238, 238, 477: 238, 238, 482: 238, 238, 238, 238, 488: 238, 490: 238, 494: 3246, 498: 3244, 3245, 3243, 3241, 238, 504: 238, 238, 722: 3242, 3240}, // 1620 {4, 4}, - {143: 4080}, - {237, 237, 483: 237, 488: 237, 2634, 237, 778: 2635, 4081}, - {1265, 1265, 483: 1265, 488: 1265, 490: 2637, 756: 2638, 799: 4082}, - {841, 841, 483: 2640, 488: 2641, 757: 2642, 816: 4083}, + {143: 4081}, + {237, 237, 483: 237, 488: 237, 2635, 237, 776: 2636, 4082}, + {1265, 1265, 483: 1265, 488: 1265, 490: 2638, 754: 2639, 797: 4083}, + {841, 841, 483: 2641, 488: 2642, 755: 2643, 814: 4084}, // 1625 {2, 2}, - {559: 4086}, + {559: 4087}, {2: 1812, 1812, 1812, 1812, 1812, 8: 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 51: 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 1812, 462: 1812, 484: 1812, 486: 1812, 559: 1812, 568: 1812}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4087}, - {2341, 2341, 2341, 2341, 4132, 4134, 389, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 69: 4103, 72: 4124, 4125, 81: 4126, 132: 4106, 192: 4091, 4110, 196: 4111, 209: 4105, 215: 4121, 226: 4100, 236: 4107, 240: 4102, 256: 4112, 264: 4108, 271: 4122, 4123, 278: 4092, 463: 4120, 468: 4131, 4168, 473: 2118, 480: 2341, 485: 4127, 490: 4109, 492: 4119, 494: 2118, 501: 4094, 574: 4099, 4095, 637: 2118, 639: 4137, 643: 4089, 652: 4114, 659: 4101, 661: 4128, 669: 4113, 676: 4115, 679: 4096, 694: 4104, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4170, 856: 4118, 869: 4116, 906: 4093, 913: 4097, 973: 4130, 1117: 4098, 1144: 4117, 1149: 4129, 4088}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4088}, + {2342, 2342, 2342, 2342, 4133, 4135, 389, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 69: 4104, 72: 4125, 4126, 81: 4127, 132: 4107, 193: 4092, 4111, 197: 4112, 209: 4106, 215: 4122, 226: 4101, 236: 4108, 240: 4103, 256: 4113, 264: 4109, 272: 4123, 4124, 279: 4093, 463: 4121, 468: 4132, 4169, 473: 2118, 480: 2342, 485: 4128, 490: 4110, 492: 4120, 4095, 495: 2118, 574: 4100, 4096, 635: 2118, 637: 4138, 641: 4090, 650: 4115, 657: 4102, 659: 4129, 667: 4114, 674: 4116, 677: 4097, 692: 4105, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4171, 854: 4119, 867: 4117, 904: 4094, 911: 4098, 971: 4131, 1116: 4099, 1143: 4118, 1148: 4130, 4089}, // 1630 - {2116, 2116, 4891, 4892, 480: 4893, 1080: 4890, 1148: 4889}, - {480: 4863}, - {158: 4855}, - {464: 1999, 468: 1999, 487: 4172, 726: 4852}, - {464: 1999, 468: 1999, 487: 4172, 726: 4849}, + {2116, 2116, 4892, 4893, 480: 4894, 1079: 4891, 1147: 4890}, + {480: 4864}, + {158: 4856}, + {464: 1999, 468: 1999, 487: 4173, 724: 4853}, + {464: 1999, 468: 1999, 487: 4173, 724: 4850}, // 1635 - {2424, 2424, 2424, 2424, 4132, 4134, 389, 2424, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 468: 4131, 4168, 473: 2118, 480: 2424, 492: 4845, 494: 2118, 637: 2118, 639: 4137, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4846}, - {385: 4835}, - {640: 4827}, - {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 462: 2346, 480: 4686, 555: 2346, 566: 2335, 574: 2335, 576: 2335, 632: 2335, 4479, 639: 2335, 665: 2335, 2335, 820: 4688, 833: 4314, 859: 4684, 890: 4685, 901: 4687}, - {2414, 2414, 2414, 2414, 7: 2414, 480: 2414}, + {2425, 2425, 2425, 2425, 4133, 4135, 389, 2425, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 468: 4132, 4169, 473: 2118, 480: 2425, 492: 4846, 495: 2118, 635: 2118, 637: 4138, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4847}, + {282: 4836}, + {638: 4828}, + {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 462: 2347, 480: 4687, 555: 2347, 566: 2336, 574: 2336, 576: 2336, 630: 2336, 4480, 637: 2336, 663: 2336, 2336, 818: 4689, 831: 4315, 857: 4685, 888: 4686, 899: 4688}, + {2415, 2415, 2415, 2415, 7: 2415, 480: 2415}, // 1640 - {2413, 2413, 2413, 2413, 7: 2413, 480: 2413}, - {480: 4682}, - {480: 4679}, - {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 480: 4658, 555: 2346, 566: 4300, 574: 4315, 576: 4657, 633: 4316, 639: 4301, 665: 4661, 815: 4660, 833: 4314, 859: 4656, 901: 4659, 984: 4662}, - {480: 4645}, + {2414, 2414, 2414, 2414, 7: 2414, 480: 2414}, + {480: 4683}, + {480: 4680}, + {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 480: 4659, 555: 2347, 566: 4301, 574: 4316, 576: 4658, 631: 4317, 637: 4302, 663: 4662, 813: 4661, 831: 4315, 857: 4657, 899: 4660, 982: 4663}, + {480: 4646}, // 1645 - {480: 4643}, - {480: 4640}, - {480: 4637}, - {18: 4634, 480: 4633}, - {18: 4630, 480: 4629}, + {480: 4644}, + {480: 4641}, + {480: 4638}, + {18: 4635, 480: 4634}, + {18: 4631, 480: 4630}, // 1650 - {480: 4619}, - {649: 4612}, - {930: 4611}, - {930: 4610}, - {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 555: 2346, 833: 4314, 859: 4606}, + {480: 4620}, + {647: 4613}, + {928: 4612}, + {928: 4611}, + {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 555: 2347, 831: 4315, 857: 4607}, // 1655 - {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 555: 2346, 833: 4314, 859: 4340}, - {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 574: 4315, 633: 4316, 639: 4313, 833: 4314, 859: 4311, 984: 4312}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 467: 4298, 487: 4172, 566: 4300, 639: 4301, 4296, 726: 4297, 815: 4299, 833: 4295}, + {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 555: 2347, 831: 4315, 857: 4341}, + {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 574: 4316, 631: 4317, 637: 4314, 831: 4315, 857: 4312, 982: 4313}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 467: 4299, 487: 4173, 566: 4301, 637: 4302, 4297, 724: 4298, 813: 4300, 831: 4296}, + {2383, 2383, 2383, 2383, 7: 2383, 480: 2383}, {2382, 2382, 2382, 2382, 7: 2382, 480: 2382}, - {2381, 2381, 2381, 2381, 7: 2381, 480: 2381}, // 1660 - {2380, 2380, 2380, 2380, 7: 2380, 480: 2380}, - {2379, 2379, 2379, 2379, 6: 388, 2379, 27: 388, 480: 2379}, - {190: 4294}, - {190: 4293}, - {2376, 2376, 2376, 2376, 7: 2376, 480: 2376}, + {2381, 2381, 2381, 2381, 7: 2381, 480: 2381}, + {2380, 2380, 2380, 2380, 6: 388, 2380, 27: 388, 480: 2380}, + {191: 4295}, + {191: 4294}, + {2377, 2377, 2377, 2377, 7: 2377, 480: 2377}, // 1665 - {2375, 2375, 2375, 2375, 7: 2375, 480: 2375}, + {2376, 2376, 2376, 2376, 7: 2376, 480: 2376}, + {2372, 2372, 2372, 2372, 7: 2372, 480: 2372}, {2371, 2371, 2371, 2371, 7: 2371, 480: 2371}, - {2370, 2370, 2370, 2370, 7: 2370, 480: 2370}, - {163: 1999, 230: 1999, 248: 1999, 250: 1999, 468: 1999, 487: 4172, 726: 4287}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 468: 1999, 487: 4172, 726: 4284}, + {163: 1999, 230: 1999, 248: 1999, 250: 1999, 468: 1999, 487: 4173, 724: 4288}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 468: 1999, 487: 4173, 724: 4285}, // 1670 - {150: 4283, 677: 4282}, - {2340, 2340, 2340, 2340, 7: 4280, 480: 2340}, - {2339, 2339, 2339, 2339, 7: 2339, 480: 2339}, - {14: 2117, 16: 2117, 21: 2117, 473: 2117, 494: 2117, 637: 2117}, - {464: 1999, 487: 4172, 726: 4278}, + {150: 4284, 675: 4283}, + {2341, 2341, 2341, 2341, 7: 4281, 480: 2341}, + {2340, 2340, 2340, 2340, 7: 2340, 480: 2340}, + {14: 2117, 16: 2117, 19: 2117, 473: 2117, 495: 2117, 635: 2117}, + {464: 1999, 487: 4173, 724: 4279}, // 1675 - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 726: 4276}, - {19: 4271, 177: 4272, 237: 4273}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 726: 4269}, - {235: 4266}, - {235: 4263}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 724: 4277}, + {20: 4272, 178: 4273, 237: 4274}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 724: 4270}, + {235: 4267}, + {235: 4264}, // 1680 - {487: 4172, 496: 1999, 726: 4261}, - {487: 4172, 496: 1999, 726: 4259}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 487: 4172, 726: 4257}, - {487: 4172, 496: 1999, 726: 4255}, - {2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 13: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 50: 2072, 461: 2072, 2072, 2072, 467: 2072, 2072, 2072, 473: 2072, 479: 2072, 2072, 491: 2072, 2072, 494: 2072, 2072, 559: 2072, 636: 2072, 2072, 639: 2072}, + {487: 4173, 497: 1999, 724: 4262}, + {487: 4173, 497: 1999, 724: 4260}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 487: 4173, 724: 4258}, + {487: 4173, 497: 1999, 724: 4256}, + {2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 13: 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 2072, 50: 2072, 461: 2072, 2072, 2072, 467: 2072, 2072, 2072, 473: 2072, 479: 2072, 2072, 491: 2072, 2072, 495: 2072, 2072, 559: 2072, 634: 2072, 2072, 637: 2072}, // 1685 - {423, 423, 423, 423, 423, 423, 423, 423, 13: 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 461: 423, 423, 423, 467: 423, 423, 423, 473: 423, 479: 423, 423, 491: 423, 423, 494: 423, 423, 559: 423, 636: 423, 423, 639: 423}, - {14: 3733, 473: 4250, 494: 3734, 637: 3732, 761: 4249}, - {6: 4243, 27: 4244}, - {487: 4172, 496: 1999, 726: 4241}, - {487: 4172, 496: 1999, 726: 4239}, + {423, 423, 423, 423, 423, 423, 423, 423, 13: 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 423, 461: 423, 423, 423, 467: 423, 423, 423, 473: 423, 479: 423, 423, 491: 423, 423, 495: 423, 423, 559: 423, 634: 423, 423, 637: 423}, + {14: 3734, 473: 4251, 495: 3735, 635: 3733, 759: 4250}, + {6: 4244, 27: 4245}, + {487: 4173, 497: 1999, 724: 4242}, + {487: 4173, 497: 1999, 724: 4240}, // 1690 - {464: 1999, 487: 4172, 726: 4237}, - {487: 4172, 496: 1999, 726: 4235}, - {487: 4172, 496: 1999, 726: 4233}, - {464: 1999, 487: 4172, 726: 4231}, - {464: 1999, 487: 4172, 726: 4229}, + {464: 1999, 487: 4173, 724: 4238}, + {487: 4173, 497: 1999, 724: 4236}, + {487: 4173, 497: 1999, 724: 4234}, + {464: 1999, 487: 4173, 724: 4232}, + {464: 1999, 487: 4173, 724: 4230}, // 1695 - {487: 4172, 496: 1999, 726: 4227}, - {487: 4172, 496: 1999, 726: 4225}, - {409, 409, 409, 409, 409, 409, 409, 409, 13: 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 461: 409, 409, 409, 467: 409, 409, 409, 473: 409, 479: 409, 409, 491: 409, 409, 494: 409, 409, 559: 409, 636: 409, 409, 639: 409}, - {468: 1999, 487: 4172, 496: 1999, 726: 4223}, - {468: 1999, 487: 4172, 496: 1999, 726: 4220}, + {487: 4173, 497: 1999, 724: 4228}, + {487: 4173, 497: 1999, 724: 4226}, + {409, 409, 409, 409, 409, 409, 409, 409, 13: 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 409, 461: 409, 409, 409, 467: 409, 409, 409, 473: 409, 479: 409, 409, 491: 409, 409, 495: 409, 409, 559: 409, 634: 409, 409, 637: 409}, + {468: 1999, 487: 4173, 497: 1999, 724: 4224}, + {468: 1999, 487: 4173, 497: 1999, 724: 4221}, // 1700 - {468: 1999, 487: 4172, 496: 1999, 726: 4217}, - {487: 4172, 496: 1999, 726: 4215}, - {487: 4172, 496: 1999, 726: 4213}, - {487: 4172, 496: 1999, 562: 1999, 1999, 726: 4211}, - {464: 1999, 487: 4172, 726: 4209}, + {468: 1999, 487: 4173, 497: 1999, 724: 4218}, + {487: 4173, 497: 1999, 724: 4216}, + {487: 4173, 497: 1999, 724: 4214}, + {487: 4173, 497: 1999, 562: 1999, 1999, 724: 4212}, + {464: 1999, 487: 4173, 724: 4210}, // 1705 - {464: 1999, 487: 4172, 726: 4207}, - {487: 4172, 496: 1999, 726: 4205}, - {487: 4172, 496: 1999, 726: 4203}, - {468: 1999, 487: 4172, 496: 1999, 726: 4199}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 481: 1999, 487: 4172, 726: 4196}, + {464: 1999, 487: 4173, 724: 4208}, + {487: 4173, 497: 1999, 724: 4206}, + {487: 4173, 497: 1999, 724: 4204}, + {468: 1999, 487: 4173, 497: 1999, 724: 4200}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 481: 1999, 487: 4173, 724: 4197}, // 1710 - {462: 1999, 487: 4172, 726: 4191}, - {464: 1999, 487: 4172, 726: 4188}, - {383, 383, 383, 383, 383, 383, 383, 383, 13: 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 461: 383, 383, 383, 467: 383, 383, 383, 473: 383, 479: 383, 383, 491: 383, 383, 494: 383, 383, 559: 383, 636: 383, 383, 639: 383}, - {173: 1999, 195: 1999, 227: 1999, 1999, 265: 1999, 281: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 468: 1999, 487: 4172, 726: 4173}, - {2: 1998, 1998, 1998, 1998, 1998, 8: 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 51: 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 462: 1998, 464: 1998, 468: 1998, 474: 1998, 1998, 479: 1998, 481: 1998, 495: 1998, 1998, 532: 1998, 560: 1998, 1998, 1998, 1998}, + {462: 1999, 487: 4173, 724: 4192}, + {464: 1999, 487: 4173, 724: 4189}, + {383, 383, 383, 383, 383, 383, 383, 383, 13: 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 383, 461: 383, 383, 383, 467: 383, 383, 383, 473: 383, 479: 383, 383, 491: 383, 383, 495: 383, 383, 559: 383, 634: 383, 383, 637: 383}, + {173: 1999, 196: 1999, 227: 1999, 1999, 265: 1999, 283: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 468: 1999, 487: 4173, 724: 4174}, + {2: 1998, 1998, 1998, 1998, 1998, 8: 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 51: 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 1998, 462: 1998, 464: 1998, 468: 1998, 474: 1998, 1998, 479: 1998, 481: 1998, 496: 1998, 1998, 534: 1998, 560: 1998, 1998, 1998, 1998}, // 1715 - {173: 4176, 195: 4175, 227: 4179, 4177, 265: 4178, 281: 4180, 4181, 4185, 4184, 4182, 4186, 4187, 4183, 468: 4174}, - {377, 377, 377, 377, 377, 377, 377, 377, 13: 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 461: 377, 377, 377, 467: 377, 377, 377, 473: 377, 479: 377, 377, 491: 377, 377, 494: 377, 377, 559: 377, 636: 377, 377, 639: 377}, - {376, 376, 376, 376, 376, 376, 376, 376, 13: 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 461: 376, 376, 376, 467: 376, 376, 376, 473: 376, 479: 376, 376, 491: 376, 376, 494: 376, 376, 559: 376, 636: 376, 376, 639: 376}, - {375, 375, 375, 375, 375, 375, 375, 375, 13: 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 461: 375, 375, 375, 467: 375, 375, 375, 473: 375, 479: 375, 375, 491: 375, 375, 494: 375, 375, 559: 375, 636: 375, 375, 639: 375}, - {374, 374, 374, 374, 374, 374, 374, 374, 13: 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 461: 374, 374, 374, 467: 374, 374, 374, 473: 374, 479: 374, 374, 491: 374, 374, 494: 374, 374, 559: 374, 636: 374, 374, 639: 374}, + {173: 4177, 196: 4176, 227: 4180, 4178, 265: 4179, 283: 4181, 4182, 4186, 4185, 4183, 4187, 4188, 4184, 468: 4175}, + {377, 377, 377, 377, 377, 377, 377, 377, 13: 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 377, 461: 377, 377, 377, 467: 377, 377, 377, 473: 377, 479: 377, 377, 491: 377, 377, 495: 377, 377, 559: 377, 634: 377, 377, 637: 377}, + {376, 376, 376, 376, 376, 376, 376, 376, 13: 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 376, 461: 376, 376, 376, 467: 376, 376, 376, 473: 376, 479: 376, 376, 491: 376, 376, 495: 376, 376, 559: 376, 634: 376, 376, 637: 376}, + {375, 375, 375, 375, 375, 375, 375, 375, 13: 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 375, 461: 375, 375, 375, 467: 375, 375, 375, 473: 375, 479: 375, 375, 491: 375, 375, 495: 375, 375, 559: 375, 634: 375, 375, 637: 375}, + {374, 374, 374, 374, 374, 374, 374, 374, 13: 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 374, 461: 374, 374, 374, 467: 374, 374, 374, 473: 374, 479: 374, 374, 491: 374, 374, 495: 374, 374, 559: 374, 634: 374, 374, 637: 374}, // 1720 - {373, 373, 373, 373, 373, 373, 373, 373, 13: 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 461: 373, 373, 373, 467: 373, 373, 373, 473: 373, 479: 373, 373, 491: 373, 373, 494: 373, 373, 559: 373, 636: 373, 373, 639: 373}, - {372, 372, 372, 372, 372, 372, 372, 372, 13: 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 461: 372, 372, 372, 467: 372, 372, 372, 473: 372, 479: 372, 372, 491: 372, 372, 494: 372, 372, 559: 372, 636: 372, 372, 639: 372}, - {371, 371, 371, 371, 371, 371, 371, 371, 13: 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 461: 371, 371, 371, 467: 371, 371, 371, 473: 371, 479: 371, 371, 491: 371, 371, 494: 371, 371, 559: 371, 636: 371, 371, 639: 371}, - {370, 370, 370, 370, 370, 370, 370, 370, 13: 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 461: 370, 370, 370, 467: 370, 370, 370, 473: 370, 479: 370, 370, 491: 370, 370, 494: 370, 370, 559: 370, 636: 370, 370, 639: 370}, - {369, 369, 369, 369, 369, 369, 369, 369, 13: 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 461: 369, 369, 369, 467: 369, 369, 369, 473: 369, 479: 369, 369, 491: 369, 369, 494: 369, 369, 559: 369, 636: 369, 369, 639: 369}, + {373, 373, 373, 373, 373, 373, 373, 373, 13: 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 373, 461: 373, 373, 373, 467: 373, 373, 373, 473: 373, 479: 373, 373, 491: 373, 373, 495: 373, 373, 559: 373, 634: 373, 373, 637: 373}, + {372, 372, 372, 372, 372, 372, 372, 372, 13: 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 372, 461: 372, 372, 372, 467: 372, 372, 372, 473: 372, 479: 372, 372, 491: 372, 372, 495: 372, 372, 559: 372, 634: 372, 372, 637: 372}, + {371, 371, 371, 371, 371, 371, 371, 371, 13: 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 371, 461: 371, 371, 371, 467: 371, 371, 371, 473: 371, 479: 371, 371, 491: 371, 371, 495: 371, 371, 559: 371, 634: 371, 371, 637: 371}, + {370, 370, 370, 370, 370, 370, 370, 370, 13: 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 370, 461: 370, 370, 370, 467: 370, 370, 370, 473: 370, 479: 370, 370, 491: 370, 370, 495: 370, 370, 559: 370, 634: 370, 370, 637: 370}, + {369, 369, 369, 369, 369, 369, 369, 369, 13: 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 369, 461: 369, 369, 369, 467: 369, 369, 369, 473: 369, 479: 369, 369, 491: 369, 369, 495: 369, 369, 559: 369, 634: 369, 369, 637: 369}, // 1725 - {368, 368, 368, 368, 368, 368, 368, 368, 13: 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 461: 368, 368, 368, 467: 368, 368, 368, 473: 368, 479: 368, 368, 491: 368, 368, 494: 368, 368, 559: 368, 636: 368, 368, 639: 368}, - {367, 367, 367, 367, 367, 367, 367, 367, 13: 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 461: 367, 367, 367, 467: 367, 367, 367, 473: 367, 479: 367, 367, 491: 367, 367, 494: 367, 367, 559: 367, 636: 367, 367, 639: 367}, - {366, 366, 366, 366, 366, 366, 366, 366, 13: 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 461: 366, 366, 366, 467: 366, 366, 366, 473: 366, 479: 366, 366, 491: 366, 366, 494: 366, 366, 559: 366, 636: 366, 366, 639: 366}, - {365, 365, 365, 365, 365, 365, 365, 365, 13: 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 461: 365, 365, 365, 467: 365, 365, 365, 473: 365, 479: 365, 365, 491: 365, 365, 494: 365, 365, 559: 365, 636: 365, 365, 639: 365}, - {364, 364, 364, 364, 364, 364, 364, 364, 13: 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 461: 364, 364, 364, 467: 364, 364, 364, 473: 364, 479: 364, 364, 491: 364, 364, 494: 364, 364, 559: 364, 636: 364, 364, 639: 364}, + {368, 368, 368, 368, 368, 368, 368, 368, 13: 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 368, 461: 368, 368, 368, 467: 368, 368, 368, 473: 368, 479: 368, 368, 491: 368, 368, 495: 368, 368, 559: 368, 634: 368, 368, 637: 368}, + {367, 367, 367, 367, 367, 367, 367, 367, 13: 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 367, 461: 367, 367, 367, 467: 367, 367, 367, 473: 367, 479: 367, 367, 491: 367, 367, 495: 367, 367, 559: 367, 634: 367, 367, 637: 367}, + {366, 366, 366, 366, 366, 366, 366, 366, 13: 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 366, 461: 366, 366, 366, 467: 366, 366, 366, 473: 366, 479: 366, 366, 491: 366, 366, 495: 366, 366, 559: 366, 634: 366, 366, 637: 366}, + {365, 365, 365, 365, 365, 365, 365, 365, 13: 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 365, 461: 365, 365, 365, 467: 365, 365, 365, 473: 365, 479: 365, 365, 491: 365, 365, 495: 365, 365, 559: 365, 634: 365, 365, 637: 365}, + {364, 364, 364, 364, 364, 364, 364, 364, 13: 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 364, 461: 364, 364, 364, 467: 364, 364, 364, 473: 364, 479: 364, 364, 491: 364, 364, 495: 364, 364, 559: 364, 634: 364, 364, 637: 364}, // 1730 - {464: 4190, 1023: 4189}, - {390, 390, 390, 390, 390, 390, 390, 390, 13: 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 461: 390, 390, 390, 467: 390, 390, 390, 473: 390, 479: 390, 390, 491: 390, 390, 494: 390, 390, 559: 390, 636: 390, 390, 639: 390}, - {9, 9, 9, 9, 9, 9, 9, 9, 13: 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 461: 9, 9, 9, 467: 9, 9, 9, 473: 9, 479: 9, 9, 491: 9, 9, 494: 9, 9, 559: 9, 636: 9, 9, 639: 9}, - {462: 4192}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 547, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 4193, 1126: 4194}, + {464: 4191, 1021: 4190}, + {390, 390, 390, 390, 390, 390, 390, 390, 13: 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 390, 461: 390, 390, 390, 467: 390, 390, 390, 473: 390, 479: 390, 390, 491: 390, 390, 495: 390, 390, 559: 390, 634: 390, 390, 637: 390}, + {9, 9, 9, 9, 9, 9, 9, 9, 13: 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 461: 9, 9, 9, 467: 9, 9, 9, 473: 9, 479: 9, 9, 491: 9, 9, 9, 495: 9, 9, 559: 9, 634: 9, 9, 637: 9}, + {462: 4193}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 547, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 4194, 1125: 4195}, // 1735 - {546, 546, 7: 3821, 50: 546, 463: 546}, - {50: 4195}, - {391, 391, 391, 391, 391, 391, 391, 391, 13: 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 461: 391, 391, 391, 467: 391, 391, 391, 473: 391, 479: 391, 391, 491: 391, 391, 494: 391, 391, 559: 391, 636: 391, 391, 639: 391}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 481: 4197, 653: 3390, 2674, 2675, 2673, 728: 4198}, - {393, 393, 393, 393, 393, 393, 393, 393, 13: 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 461: 393, 393, 393, 467: 393, 393, 393, 473: 393, 479: 393, 393, 491: 393, 393, 494: 393, 393, 559: 393, 636: 393, 393, 639: 393}, + {546, 546, 7: 3822, 50: 546, 463: 546}, + {50: 4196}, + {391, 391, 391, 391, 391, 391, 391, 391, 13: 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 391, 461: 391, 391, 391, 467: 391, 391, 391, 473: 391, 479: 391, 391, 491: 391, 391, 495: 391, 391, 559: 391, 634: 391, 391, 637: 391}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 481: 4198, 651: 3391, 2675, 2676, 2674, 726: 4199}, + {393, 393, 393, 393, 393, 393, 393, 393, 13: 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 393, 461: 393, 393, 393, 467: 393, 393, 393, 473: 393, 479: 393, 393, 491: 393, 393, 495: 393, 393, 559: 393, 634: 393, 393, 637: 393}, // 1740 - {392, 392, 392, 392, 392, 392, 392, 392, 13: 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 461: 392, 392, 392, 467: 392, 392, 392, 473: 392, 479: 392, 392, 491: 392, 392, 494: 392, 392, 559: 392, 636: 392, 392, 639: 392}, - {468: 4201, 496: 2648, 725: 2647, 734: 4202, 1118: 4200}, - {396, 396, 396, 396, 396, 396, 396, 396, 13: 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 461: 396, 396, 396, 467: 396, 396, 396, 473: 396, 479: 396, 396, 491: 396, 396, 494: 396, 396, 559: 396, 636: 396, 396, 639: 396}, - {387, 387, 387, 387, 387, 387, 387, 387, 13: 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 461: 387, 387, 387, 467: 387, 387, 387, 473: 387, 479: 387, 387, 491: 387, 387, 494: 387, 387, 559: 387, 636: 387, 387, 639: 387}, - {386, 386, 386, 386, 386, 386, 386, 386, 13: 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 461: 386, 386, 386, 467: 386, 386, 386, 473: 386, 479: 386, 386, 491: 386, 386, 494: 386, 386, 559: 386, 636: 386, 386, 639: 386}, + {392, 392, 392, 392, 392, 392, 392, 392, 13: 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 392, 461: 392, 392, 392, 467: 392, 392, 392, 473: 392, 479: 392, 392, 491: 392, 392, 495: 392, 392, 559: 392, 634: 392, 392, 637: 392}, + {468: 4202, 497: 2649, 721: 2648, 731: 4203, 1117: 4201}, + {396, 396, 396, 396, 396, 396, 396, 396, 13: 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 396, 461: 396, 396, 396, 467: 396, 396, 396, 473: 396, 479: 396, 396, 491: 396, 396, 495: 396, 396, 559: 396, 634: 396, 396, 637: 396}, + {387, 387, 387, 387, 387, 387, 387, 387, 13: 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 387, 461: 387, 387, 387, 467: 387, 387, 387, 473: 387, 479: 387, 387, 491: 387, 387, 495: 387, 387, 559: 387, 634: 387, 387, 637: 387}, + {386, 386, 386, 386, 386, 386, 386, 386, 13: 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 386, 461: 386, 386, 386, 467: 386, 386, 386, 473: 386, 479: 386, 386, 491: 386, 386, 495: 386, 386, 559: 386, 634: 386, 386, 637: 386}, // 1745 - {496: 2648, 725: 2647, 734: 4204}, - {397, 397, 397, 397, 397, 397, 397, 397, 13: 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 461: 397, 397, 397, 467: 397, 397, 397, 473: 397, 479: 397, 397, 491: 397, 397, 494: 397, 397, 559: 397, 636: 397, 397, 639: 397}, - {496: 2648, 725: 2647, 734: 4206}, - {398, 398, 398, 398, 398, 398, 398, 398, 13: 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 461: 398, 398, 398, 467: 398, 398, 398, 473: 398, 479: 398, 398, 491: 398, 398, 494: 398, 398, 559: 398, 636: 398, 398, 639: 398}, - {464: 4208}, + {497: 2649, 721: 2648, 731: 4205}, + {397, 397, 397, 397, 397, 397, 397, 397, 13: 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 397, 461: 397, 397, 397, 467: 397, 397, 397, 473: 397, 479: 397, 397, 491: 397, 397, 495: 397, 397, 559: 397, 634: 397, 397, 637: 397}, + {497: 2649, 721: 2648, 731: 4207}, + {398, 398, 398, 398, 398, 398, 398, 398, 13: 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 398, 461: 398, 398, 398, 467: 398, 398, 398, 473: 398, 479: 398, 398, 491: 398, 398, 495: 398, 398, 559: 398, 634: 398, 398, 637: 398}, + {464: 4209}, // 1750 - {399, 399, 399, 399, 399, 399, 399, 399, 13: 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 461: 399, 399, 399, 467: 399, 399, 399, 473: 399, 479: 399, 399, 491: 399, 399, 494: 399, 399, 559: 399, 636: 399, 399, 639: 399}, - {464: 4210}, - {400, 400, 400, 400, 400, 400, 400, 400, 13: 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 461: 400, 400, 400, 467: 400, 400, 400, 473: 400, 479: 400, 400, 491: 400, 400, 494: 400, 400, 559: 400, 636: 400, 400, 639: 400}, - {496: 3322, 562: 3324, 3323, 807: 4212}, - {401, 401, 401, 401, 401, 401, 401, 401, 13: 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 461: 401, 401, 401, 467: 401, 401, 401, 473: 401, 479: 401, 401, 491: 401, 401, 494: 401, 401, 559: 401, 636: 401, 401, 639: 401}, + {399, 399, 399, 399, 399, 399, 399, 399, 13: 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 399, 461: 399, 399, 399, 467: 399, 399, 399, 473: 399, 479: 399, 399, 491: 399, 399, 495: 399, 399, 559: 399, 634: 399, 399, 637: 399}, + {464: 4211}, + {400, 400, 400, 400, 400, 400, 400, 400, 13: 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 400, 461: 400, 400, 400, 467: 400, 400, 400, 473: 400, 479: 400, 400, 491: 400, 400, 495: 400, 400, 559: 400, 634: 400, 400, 637: 400}, + {497: 3323, 562: 3325, 3324, 805: 4213}, + {401, 401, 401, 401, 401, 401, 401, 401, 13: 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 401, 461: 401, 401, 401, 467: 401, 401, 401, 473: 401, 479: 401, 401, 491: 401, 401, 495: 401, 401, 559: 401, 634: 401, 401, 637: 401}, // 1755 - {496: 2648, 725: 2647, 734: 4214}, - {402, 402, 402, 402, 402, 402, 402, 402, 13: 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 461: 402, 402, 402, 467: 402, 402, 402, 473: 402, 479: 402, 402, 491: 402, 402, 494: 402, 402, 559: 402, 636: 402, 402, 639: 402}, - {496: 2648, 725: 2647, 734: 4216}, - {403, 403, 403, 403, 403, 403, 403, 403, 13: 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 461: 403, 403, 403, 467: 403, 403, 403, 473: 403, 479: 403, 403, 491: 403, 403, 494: 403, 403, 559: 403, 636: 403, 403, 639: 403}, - {468: 4219, 496: 2648, 725: 2647, 734: 4218}, + {497: 2649, 721: 2648, 731: 4215}, + {402, 402, 402, 402, 402, 402, 402, 402, 13: 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 402, 461: 402, 402, 402, 467: 402, 402, 402, 473: 402, 479: 402, 402, 491: 402, 402, 495: 402, 402, 559: 402, 634: 402, 402, 637: 402}, + {497: 2649, 721: 2648, 731: 4217}, + {403, 403, 403, 403, 403, 403, 403, 403, 13: 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 403, 461: 403, 403, 403, 467: 403, 403, 403, 473: 403, 479: 403, 403, 491: 403, 403, 495: 403, 403, 559: 403, 634: 403, 403, 637: 403}, + {468: 4220, 497: 2649, 721: 2648, 731: 4219}, // 1760 - {405, 405, 405, 405, 405, 405, 405, 405, 13: 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 461: 405, 405, 405, 467: 405, 405, 405, 473: 405, 479: 405, 405, 491: 405, 405, 494: 405, 405, 559: 405, 636: 405, 405, 639: 405}, - {404, 404, 404, 404, 404, 404, 404, 404, 13: 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 461: 404, 404, 404, 467: 404, 404, 404, 473: 404, 479: 404, 404, 491: 404, 404, 494: 404, 404, 559: 404, 636: 404, 404, 639: 404}, - {468: 4222, 496: 2648, 725: 2647, 734: 4221}, - {407, 407, 407, 407, 407, 407, 407, 407, 13: 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 461: 407, 407, 407, 467: 407, 407, 407, 473: 407, 479: 407, 407, 491: 407, 407, 494: 407, 407, 559: 407, 636: 407, 407, 639: 407}, - {406, 406, 406, 406, 406, 406, 406, 406, 13: 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 461: 406, 406, 406, 467: 406, 406, 406, 473: 406, 479: 406, 406, 491: 406, 406, 494: 406, 406, 559: 406, 636: 406, 406, 639: 406}, + {405, 405, 405, 405, 405, 405, 405, 405, 13: 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 405, 461: 405, 405, 405, 467: 405, 405, 405, 473: 405, 479: 405, 405, 491: 405, 405, 495: 405, 405, 559: 405, 634: 405, 405, 637: 405}, + {404, 404, 404, 404, 404, 404, 404, 404, 13: 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 404, 461: 404, 404, 404, 467: 404, 404, 404, 473: 404, 479: 404, 404, 491: 404, 404, 495: 404, 404, 559: 404, 634: 404, 404, 637: 404}, + {468: 4223, 497: 2649, 721: 2648, 731: 4222}, + {407, 407, 407, 407, 407, 407, 407, 407, 13: 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 407, 461: 407, 407, 407, 467: 407, 407, 407, 473: 407, 479: 407, 407, 491: 407, 407, 495: 407, 407, 559: 407, 634: 407, 407, 637: 407}, + {406, 406, 406, 406, 406, 406, 406, 406, 13: 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 406, 461: 406, 406, 406, 467: 406, 406, 406, 473: 406, 479: 406, 406, 491: 406, 406, 495: 406, 406, 559: 406, 634: 406, 406, 637: 406}, // 1765 - {468: 4201, 496: 2648, 725: 2647, 734: 4202, 1118: 4224}, - {408, 408, 408, 408, 408, 408, 408, 408, 13: 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 461: 408, 408, 408, 467: 408, 408, 408, 473: 408, 479: 408, 408, 491: 408, 408, 494: 408, 408, 559: 408, 636: 408, 408, 639: 408}, - {496: 2648, 725: 2647, 734: 4226}, - {410, 410, 410, 410, 410, 410, 410, 410, 13: 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 461: 410, 410, 410, 467: 410, 410, 410, 473: 410, 479: 410, 410, 491: 410, 410, 494: 410, 410, 559: 410, 636: 410, 410, 639: 410}, - {496: 2648, 725: 2647, 734: 4228}, + {468: 4202, 497: 2649, 721: 2648, 731: 4203, 1117: 4225}, + {408, 408, 408, 408, 408, 408, 408, 408, 13: 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 408, 461: 408, 408, 408, 467: 408, 408, 408, 473: 408, 479: 408, 408, 491: 408, 408, 495: 408, 408, 559: 408, 634: 408, 408, 637: 408}, + {497: 2649, 721: 2648, 731: 4227}, + {410, 410, 410, 410, 410, 410, 410, 410, 13: 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 410, 461: 410, 410, 410, 467: 410, 410, 410, 473: 410, 479: 410, 410, 491: 410, 410, 495: 410, 410, 559: 410, 634: 410, 410, 637: 410}, + {497: 2649, 721: 2648, 731: 4229}, // 1770 - {411, 411, 411, 411, 411, 411, 411, 411, 13: 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 461: 411, 411, 411, 467: 411, 411, 411, 473: 411, 479: 411, 411, 491: 411, 411, 494: 411, 411, 559: 411, 636: 411, 411, 639: 411}, - {464: 4230}, - {412, 412, 412, 412, 412, 412, 412, 412, 13: 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 461: 412, 412, 412, 467: 412, 412, 412, 473: 412, 479: 412, 412, 491: 412, 412, 494: 412, 412, 559: 412, 636: 412, 412, 639: 412}, - {464: 4232}, - {413, 413, 413, 413, 413, 413, 413, 413, 13: 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 461: 413, 413, 413, 467: 413, 413, 413, 473: 413, 479: 413, 413, 491: 413, 413, 494: 413, 413, 559: 413, 636: 413, 413, 639: 413}, + {411, 411, 411, 411, 411, 411, 411, 411, 13: 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 411, 461: 411, 411, 411, 467: 411, 411, 411, 473: 411, 479: 411, 411, 491: 411, 411, 495: 411, 411, 559: 411, 634: 411, 411, 637: 411}, + {464: 4231}, + {412, 412, 412, 412, 412, 412, 412, 412, 13: 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 412, 461: 412, 412, 412, 467: 412, 412, 412, 473: 412, 479: 412, 412, 491: 412, 412, 495: 412, 412, 559: 412, 634: 412, 412, 637: 412}, + {464: 4233}, + {413, 413, 413, 413, 413, 413, 413, 413, 13: 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 413, 461: 413, 413, 413, 467: 413, 413, 413, 473: 413, 479: 413, 413, 491: 413, 413, 495: 413, 413, 559: 413, 634: 413, 413, 637: 413}, // 1775 - {496: 2648, 725: 2647, 734: 4234}, - {414, 414, 414, 414, 414, 414, 414, 414, 13: 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 461: 414, 414, 414, 467: 414, 414, 414, 473: 414, 479: 414, 414, 491: 414, 414, 494: 414, 414, 559: 414, 636: 414, 414, 639: 414}, - {496: 2648, 725: 2647, 734: 4236}, - {415, 415, 415, 415, 415, 415, 415, 415, 13: 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 461: 415, 415, 415, 467: 415, 415, 415, 473: 415, 479: 415, 415, 491: 415, 415, 494: 415, 415, 559: 415, 636: 415, 415, 639: 415}, - {464: 4238}, + {497: 2649, 721: 2648, 731: 4235}, + {414, 414, 414, 414, 414, 414, 414, 414, 13: 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 414, 461: 414, 414, 414, 467: 414, 414, 414, 473: 414, 479: 414, 414, 491: 414, 414, 495: 414, 414, 559: 414, 634: 414, 414, 637: 414}, + {497: 2649, 721: 2648, 731: 4237}, + {415, 415, 415, 415, 415, 415, 415, 415, 13: 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 415, 461: 415, 415, 415, 467: 415, 415, 415, 473: 415, 479: 415, 415, 491: 415, 415, 495: 415, 415, 559: 415, 634: 415, 415, 637: 415}, + {464: 4239}, // 1780 - {416, 416, 416, 416, 416, 416, 416, 416, 13: 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 461: 416, 416, 416, 467: 416, 416, 416, 473: 416, 479: 416, 416, 491: 416, 416, 494: 416, 416, 559: 416, 636: 416, 416, 639: 416}, - {496: 2648, 725: 2647, 734: 4240}, - {417, 417, 417, 417, 417, 417, 417, 417, 13: 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 461: 417, 417, 417, 467: 417, 417, 417, 473: 417, 479: 417, 417, 491: 417, 417, 494: 417, 417, 559: 417, 636: 417, 417, 639: 417}, - {496: 2648, 725: 2647, 734: 4242}, - {419, 419, 419, 419, 419, 419, 419, 419, 13: 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 461: 419, 419, 419, 467: 419, 419, 419, 473: 419, 479: 419, 419, 491: 419, 419, 494: 419, 419, 559: 419, 636: 419, 419, 639: 419}, + {416, 416, 416, 416, 416, 416, 416, 416, 13: 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 416, 461: 416, 416, 416, 467: 416, 416, 416, 473: 416, 479: 416, 416, 491: 416, 416, 495: 416, 416, 559: 416, 634: 416, 416, 637: 416}, + {497: 2649, 721: 2648, 731: 4241}, + {417, 417, 417, 417, 417, 417, 417, 417, 13: 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 417, 461: 417, 417, 417, 467: 417, 417, 417, 473: 417, 479: 417, 417, 491: 417, 417, 495: 417, 417, 559: 417, 634: 417, 417, 637: 417}, + {497: 2649, 721: 2648, 731: 4243}, + {419, 419, 419, 419, 419, 419, 419, 419, 13: 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 419, 461: 419, 419, 419, 467: 419, 419, 419, 473: 419, 479: 419, 419, 491: 419, 419, 495: 419, 419, 559: 419, 634: 419, 419, 637: 419}, // 1785 - {487: 4172, 496: 1999, 726: 4247}, - {487: 4172, 496: 1999, 726: 4245}, - {496: 2648, 725: 2647, 734: 4246}, - {418, 418, 418, 418, 418, 418, 418, 418, 13: 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 461: 418, 418, 418, 467: 418, 418, 418, 473: 418, 479: 418, 418, 491: 418, 418, 494: 418, 418, 559: 418, 636: 418, 418, 639: 418}, - {496: 2648, 725: 2647, 734: 4248}, + {487: 4173, 497: 1999, 724: 4248}, + {487: 4173, 497: 1999, 724: 4246}, + {497: 2649, 721: 2648, 731: 4247}, + {418, 418, 418, 418, 418, 418, 418, 418, 13: 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 418, 461: 418, 418, 418, 467: 418, 418, 418, 473: 418, 479: 418, 418, 491: 418, 418, 495: 418, 418, 559: 418, 634: 418, 418, 637: 418}, + {497: 2649, 721: 2648, 731: 4249}, // 1790 - {420, 420, 420, 420, 420, 420, 420, 420, 13: 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 461: 420, 420, 420, 467: 420, 420, 420, 473: 420, 479: 420, 420, 491: 420, 420, 494: 420, 420, 559: 420, 636: 420, 420, 639: 420}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 532: 1999, 726: 4253}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 532: 1999, 726: 4251}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3388, 653: 3390, 2674, 2675, 2673, 728: 3387, 858: 4252}, - {421, 421, 421, 421, 421, 421, 421, 421, 13: 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 461: 421, 421, 421, 467: 421, 421, 421, 473: 421, 479: 421, 421, 491: 421, 421, 494: 421, 421, 559: 421, 636: 421, 421, 639: 421}, + {420, 420, 420, 420, 420, 420, 420, 420, 13: 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 420, 461: 420, 420, 420, 467: 420, 420, 420, 473: 420, 479: 420, 420, 491: 420, 420, 495: 420, 420, 559: 420, 634: 420, 420, 637: 420}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 534: 1999, 724: 4254}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 534: 1999, 724: 4252}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3389, 651: 3391, 2675, 2676, 2674, 726: 3388, 856: 4253}, + {421, 421, 421, 421, 421, 421, 421, 421, 13: 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 421, 461: 421, 421, 421, 467: 421, 421, 421, 473: 421, 479: 421, 421, 491: 421, 421, 495: 421, 421, 559: 421, 634: 421, 421, 637: 421}, // 1795 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 4254}, - {422, 422, 422, 422, 422, 422, 422, 422, 13: 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 461: 422, 422, 422, 467: 422, 422, 422, 473: 422, 479: 422, 422, 491: 422, 422, 494: 422, 422, 559: 422, 636: 422, 422, 639: 422}, - {496: 2648, 725: 2647, 734: 4256}, - {2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 13: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 50: 2073, 461: 2073, 2073, 2073, 467: 2073, 2073, 2073, 473: 2073, 479: 2073, 2073, 491: 2073, 2073, 494: 2073, 2073, 559: 2073, 636: 2073, 2073, 639: 2073}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4258, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 4255}, + {422, 422, 422, 422, 422, 422, 422, 422, 13: 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 422, 461: 422, 422, 422, 467: 422, 422, 422, 473: 422, 479: 422, 422, 491: 422, 422, 495: 422, 422, 559: 422, 634: 422, 422, 637: 422}, + {497: 2649, 721: 2648, 731: 4257}, + {2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 13: 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 2073, 50: 2073, 461: 2073, 2073, 2073, 467: 2073, 2073, 2073, 473: 2073, 479: 2073, 2073, 491: 2073, 2073, 495: 2073, 2073, 559: 2073, 634: 2073, 2073, 637: 2073}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4259, 2675, 2676, 2674}, // 1800 - {2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 13: 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 50: 2074, 461: 2074, 2074, 2074, 467: 2074, 2074, 2074, 473: 2074, 479: 2074, 2074, 491: 2074, 2074, 494: 2074, 2074, 559: 2074, 636: 2074, 2074, 639: 2074}, - {496: 2648, 725: 2647, 734: 4260}, - {2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 13: 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 50: 2075, 461: 2075, 2075, 2075, 467: 2075, 2075, 2075, 473: 2075, 479: 2075, 2075, 491: 2075, 2075, 494: 2075, 2075, 559: 2075, 636: 2075, 2075, 639: 2075}, - {496: 2648, 725: 2647, 734: 4262}, - {2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 13: 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 50: 2076, 461: 2076, 2076, 2076, 467: 2076, 2076, 2076, 473: 2076, 479: 2076, 2076, 491: 2076, 2076, 494: 2076, 2076, 559: 2076, 636: 2076, 2076, 639: 2076}, + {2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 13: 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 2074, 50: 2074, 461: 2074, 2074, 2074, 467: 2074, 2074, 2074, 473: 2074, 479: 2074, 2074, 491: 2074, 2074, 495: 2074, 2074, 559: 2074, 634: 2074, 2074, 637: 2074}, + {497: 2649, 721: 2648, 731: 4261}, + {2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 13: 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 2075, 50: 2075, 461: 2075, 2075, 2075, 467: 2075, 2075, 2075, 473: 2075, 479: 2075, 2075, 491: 2075, 2075, 495: 2075, 2075, 559: 2075, 634: 2075, 2075, 637: 2075}, + {497: 2649, 721: 2648, 731: 4263}, + {2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 13: 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 2076, 50: 2076, 461: 2076, 2076, 2076, 467: 2076, 2076, 2076, 473: 2076, 479: 2076, 2076, 491: 2076, 2076, 495: 2076, 2076, 559: 2076, 634: 2076, 2076, 637: 2076}, // 1805 - {464: 1999, 487: 4172, 726: 4264}, - {464: 4265}, - {2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 13: 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 50: 2077, 461: 2077, 2077, 2077, 467: 2077, 2077, 2077, 473: 2077, 479: 2077, 2077, 491: 2077, 2077, 494: 2077, 2077, 559: 2077, 636: 2077, 2077, 639: 2077}, - {464: 1999, 487: 4172, 726: 4267}, - {464: 4268}, + {464: 1999, 487: 4173, 724: 4265}, + {464: 4266}, + {2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 13: 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 2077, 50: 2077, 461: 2077, 2077, 2077, 467: 2077, 2077, 2077, 473: 2077, 479: 2077, 2077, 491: 2077, 2077, 495: 2077, 2077, 559: 2077, 634: 2077, 2077, 637: 2077}, + {464: 1999, 487: 4173, 724: 4268}, + {464: 4269}, // 1810 - {2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 13: 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 50: 2078, 461: 2078, 2078, 2078, 467: 2078, 2078, 2078, 473: 2078, 479: 2078, 2078, 491: 2078, 2078, 494: 2078, 2078, 559: 2078, 636: 2078, 2078, 639: 2078}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 4270}, - {2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 13: 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 50: 2079, 461: 2079, 2079, 2079, 467: 2079, 2079, 2079, 473: 2079, 479: 2079, 2079, 491: 2079, 2079, 494: 2079, 2079, 559: 2079, 636: 2079, 2079, 639: 2079}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 726: 4274}, - {395, 395, 395, 395, 395, 395, 395, 395, 13: 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 461: 395, 395, 395, 467: 395, 395, 395, 473: 395, 479: 395, 395, 491: 395, 395, 494: 395, 395, 559: 395, 636: 395, 395, 639: 395}, + {2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 13: 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 2078, 50: 2078, 461: 2078, 2078, 2078, 467: 2078, 2078, 2078, 473: 2078, 479: 2078, 2078, 491: 2078, 2078, 495: 2078, 2078, 559: 2078, 634: 2078, 2078, 637: 2078}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 4271}, + {2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 13: 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 2079, 50: 2079, 461: 2079, 2079, 2079, 467: 2079, 2079, 2079, 473: 2079, 479: 2079, 2079, 491: 2079, 2079, 495: 2079, 2079, 559: 2079, 634: 2079, 2079, 637: 2079}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 724: 4275}, + {395, 395, 395, 395, 395, 395, 395, 395, 13: 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 395, 461: 395, 395, 395, 467: 395, 395, 395, 473: 395, 479: 395, 395, 491: 395, 395, 495: 395, 395, 559: 395, 634: 395, 395, 637: 395}, // 1815 - {394, 394, 394, 394, 394, 394, 394, 394, 13: 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 461: 394, 394, 394, 467: 394, 394, 394, 473: 394, 479: 394, 394, 491: 394, 394, 494: 394, 394, 559: 394, 636: 394, 394, 639: 394}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 4275}, - {2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 13: 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 50: 2080, 461: 2080, 2080, 2080, 467: 2080, 2080, 2080, 473: 2080, 479: 2080, 2080, 491: 2080, 2080, 494: 2080, 2080, 559: 2080, 636: 2080, 2080, 639: 2080}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 4277}, - {2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 13: 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 50: 2081, 461: 2081, 2081, 2081, 467: 2081, 2081, 2081, 473: 2081, 479: 2081, 2081, 491: 2081, 2081, 494: 2081, 2081, 559: 2081, 636: 2081, 2081, 639: 2081}, + {394, 394, 394, 394, 394, 394, 394, 394, 13: 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 394, 461: 394, 394, 394, 467: 394, 394, 394, 473: 394, 479: 394, 394, 491: 394, 394, 495: 394, 394, 559: 394, 634: 394, 394, 637: 394}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 4276}, + {2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 13: 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 2080, 50: 2080, 461: 2080, 2080, 2080, 467: 2080, 2080, 2080, 473: 2080, 479: 2080, 2080, 491: 2080, 2080, 495: 2080, 2080, 559: 2080, 634: 2080, 2080, 637: 2080}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 4278}, + {2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 13: 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 2081, 50: 2081, 461: 2081, 2081, 2081, 467: 2081, 2081, 2081, 473: 2081, 479: 2081, 2081, 491: 2081, 2081, 495: 2081, 2081, 559: 2081, 634: 2081, 2081, 637: 2081}, // 1820 - {464: 4279}, - {2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 13: 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 50: 2082, 461: 2082, 2082, 2082, 467: 2082, 2082, 2082, 473: 2082, 479: 2082, 2082, 491: 2082, 2082, 494: 2082, 2082, 559: 2082, 636: 2082, 2082, 639: 2082}, - {4: 4132, 4134, 389, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 69: 4103, 72: 4124, 4125, 81: 4126, 132: 4106, 192: 4091, 4110, 196: 4111, 209: 4105, 215: 4121, 226: 4100, 236: 4107, 240: 4102, 256: 4112, 264: 4108, 271: 4122, 4123, 278: 4092, 463: 4120, 468: 4131, 4168, 473: 2118, 485: 4127, 490: 4109, 492: 4119, 494: 2118, 501: 4094, 574: 4099, 4095, 637: 2118, 639: 4137, 652: 4114, 659: 4101, 661: 4128, 669: 4113, 676: 4115, 679: 4096, 694: 4104, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4170, 856: 4118, 869: 4116, 906: 4093, 913: 4097, 973: 4281, 1117: 4098, 1144: 4117}, - {2338, 2338, 2338, 2338, 7: 2338, 480: 2338}, - {2352, 2352, 2352, 2352, 7: 2352, 480: 2352}, + {464: 4280}, + {2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 13: 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 2082, 50: 2082, 461: 2082, 2082, 2082, 467: 2082, 2082, 2082, 473: 2082, 479: 2082, 2082, 491: 2082, 2082, 495: 2082, 2082, 559: 2082, 634: 2082, 2082, 637: 2082}, + {4: 4133, 4135, 389, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 69: 4104, 72: 4125, 4126, 81: 4127, 132: 4107, 193: 4092, 4111, 197: 4112, 209: 4106, 215: 4122, 226: 4101, 236: 4108, 240: 4103, 256: 4113, 264: 4109, 272: 4123, 4124, 279: 4093, 463: 4121, 468: 4132, 4169, 473: 2118, 485: 4128, 490: 4110, 492: 4120, 4095, 495: 2118, 574: 4100, 4096, 635: 2118, 637: 4138, 650: 4115, 657: 4102, 659: 4129, 667: 4114, 674: 4116, 677: 4097, 692: 4105, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4171, 854: 4119, 867: 4117, 904: 4094, 911: 4098, 971: 4282, 1116: 4099, 1143: 4118}, + {2339, 2339, 2339, 2339, 7: 2339, 480: 2339}, + {2353, 2353, 2353, 2353, 7: 2353, 480: 2353}, // 1825 - {2351, 2351, 2351, 2351, 7: 2351, 480: 2351}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 468: 4285, 653: 4286, 2674, 2675, 2673}, + {2352, 2352, 2352, 2352, 7: 2352, 480: 2352}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 468: 4286, 651: 4287, 2675, 2676, 2674}, + {2355, 2355, 2355, 2355, 7: 2355, 81: 2355, 480: 2355}, {2354, 2354, 2354, 2354, 7: 2354, 81: 2354, 480: 2354}, - {2353, 2353, 2353, 2353, 7: 2353, 81: 2353, 480: 2353}, - {163: 4292, 230: 4289, 248: 4290, 250: 4291, 468: 4288}, + {163: 4293, 230: 4290, 248: 4291, 250: 4292, 468: 4289}, // 1830 + {2360, 2360, 2360, 2360, 7: 2360, 480: 2360, 485: 2360}, {2359, 2359, 2359, 2359, 7: 2359, 480: 2359, 485: 2359}, {2358, 2358, 2358, 2358, 7: 2358, 480: 2358, 485: 2358}, {2357, 2357, 2357, 2357, 7: 2357, 480: 2357, 485: 2357}, {2356, 2356, 2356, 2356, 7: 2356, 480: 2356, 485: 2356}, - {2355, 2355, 2355, 2355, 7: 2355, 480: 2355, 485: 2355}, // 1835 - {2377, 2377, 2377, 2377, 7: 2377, 480: 2377}, {2378, 2378, 2378, 2378, 7: 2378, 480: 2378}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4308, 2674, 2675, 2673}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4307}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4306}, + {2379, 2379, 2379, 2379, 7: 2379, 480: 2379}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4309, 2675, 2676, 2674}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4308}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4307}, // 1840 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4305}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4302, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4306}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4303, 2675, 2676, 2674}, + {2: 2351, 2351, 2351, 2351, 2351, 8: 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 51: 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 2351, 462: 2351, 470: 2351, 482: 2351, 555: 2351}, {2: 2350, 2350, 2350, 2350, 2350, 8: 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 51: 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 2350, 462: 2350, 470: 2350, 482: 2350, 555: 2350}, - {2: 2349, 2349, 2349, 2349, 2349, 8: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 51: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 462: 2349, 470: 2349, 482: 2349, 555: 2349}, - {640: 4303}, + {638: 4304}, // 1845 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4304, 2674, 2675, 2673}, - {2383, 2383, 2383, 2383, 7: 2383, 480: 2383}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4305, 2675, 2676, 2674}, {2384, 2384, 2384, 2384, 7: 2384, 480: 2384}, {2385, 2385, 2385, 2385, 7: 2385, 480: 2385}, {2386, 2386, 2386, 2386, 7: 2386, 480: 2386}, - // 1850 - {640: 4309}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4310, 2674, 2675, 2673}, {2387, 2387, 2387, 2387, 7: 2387, 480: 2387}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4326}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4321, 2674, 2675, 2673}, + // 1850 + {638: 4310}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4311, 2675, 2676, 2674}, + {2388, 2388, 2388, 2388, 7: 2388, 480: 2388}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4327}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4322, 2675, 2676, 2674}, // 1855 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4317, 2674, 2675, 2673}, - {2: 2345, 2345, 2345, 2345, 2345, 8: 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 51: 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 2345, 462: 2345, 555: 2345}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4318, 2675, 2676, 2674}, + {2: 2346, 2346, 2346, 2346, 2346, 8: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 51: 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 2346, 462: 2346, 555: 2346}, {2: 431, 431, 431, 431, 431, 8: 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 51: 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431, 431}, {2: 430, 430, 430, 430, 430, 8: 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 51: 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430, 430}, - {84: 4320, 87: 4319, 843: 4318}, + {84: 4321, 87: 4320, 841: 4319}, // 1860 - {2372, 2372, 2372, 2372, 7: 2372, 480: 2372}, + {2373, 2373, 2373, 2373, 7: 2373, 480: 2373}, {1791, 1791, 1791, 1791, 1791, 7: 1791, 17: 1791, 50: 1791, 81: 1791, 1791, 1791, 1791, 1791, 87: 1791, 463: 1791, 470: 1791, 480: 1791, 485: 1791}, {1790, 1790, 1790, 1790, 1790, 7: 1790, 17: 1790, 50: 1790, 81: 1790, 1790, 1790, 1790, 1790, 87: 1790, 463: 1790, 470: 1790, 480: 1790, 485: 1790}, - {147: 4323, 465: 3984, 3983, 798: 4324, 921: 4322}, - {2374, 2374, 2374, 2374, 7: 2374, 480: 2374}, + {147: 4324, 465: 3985, 3984, 796: 4325, 919: 4323}, + {2375, 2375, 2375, 2375, 7: 2375, 480: 2375}, // 1865 - {2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 50: 2242, 461: 2242, 465: 2242, 2242, 2242, 2242, 473: 2242, 480: 2242, 2242, 566: 2242, 574: 2242, 576: 2242, 632: 2242, 2242, 2242, 2242}, - {147: 4325}, - {2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 50: 2241, 461: 2241, 465: 2241, 2241, 2241, 2241, 473: 2241, 480: 2241, 2241, 566: 2241, 574: 2241, 576: 2241, 632: 2241, 2241, 2241, 2241}, - {501: 4327, 659: 4328}, - {468: 4330}, + {2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 2243, 50: 2243, 461: 2243, 465: 2243, 2243, 2243, 2243, 473: 2243, 480: 2243, 2243, 566: 2243, 574: 2243, 576: 2243, 630: 2243, 2243, 2243, 2243}, + {147: 4326}, + {2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 2242, 50: 2242, 461: 2242, 465: 2242, 2242, 2242, 2242, 473: 2242, 480: 2242, 2242, 566: 2242, 574: 2242, 576: 2242, 630: 2242, 2242, 2242, 2242}, + {493: 4328, 657: 4329}, + {468: 4331}, // 1870 - {468: 4329}, - {2388, 2388, 2388, 2388, 7: 2388, 480: 2388}, - {462: 4332, 464: 3140, 474: 4335, 4334, 481: 3131, 496: 3135, 560: 3130, 3132, 3134, 3133, 565: 3138, 569: 3139, 582: 3137, 702: 4333, 3136, 1113: 4331}, - {2390, 2390, 2390, 2390, 7: 2390, 480: 2390}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4338}, + {468: 4330}, + {2389, 2389, 2389, 2389, 7: 2389, 480: 2389}, + {462: 4333, 464: 3141, 474: 4336, 4335, 481: 3132, 497: 3136, 560: 3131, 3133, 3135, 3134, 565: 3139, 569: 3140, 582: 3138, 700: 4334, 3137, 1112: 4332}, + {2391, 2391, 2391, 2391, 7: 2391, 480: 2391}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4339}, // 1875 - {2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 50: 2164, 461: 2164, 465: 2164, 2164, 2164, 2164, 473: 2164, 480: 2164, 2164, 566: 2164, 574: 2164, 576: 2164, 632: 2164, 2164, 2164, 2164}, - {496: 3322, 562: 3324, 3323, 807: 4337}, - {496: 3322, 562: 3324, 3323, 807: 4336}, - {2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 2162, 50: 2162, 461: 2162, 465: 2162, 2162, 2162, 2162, 473: 2162, 480: 2162, 2162, 566: 2162, 574: 2162, 576: 2162, 632: 2162, 2162, 2162, 2162}, - {2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 50: 2163, 461: 2163, 465: 2163, 2163, 2163, 2163, 473: 2163, 480: 2163, 2163, 566: 2163, 574: 2163, 576: 2163, 632: 2163, 2163, 2163, 2163}, + {2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 50: 2165, 461: 2165, 465: 2165, 2165, 2165, 2165, 473: 2165, 480: 2165, 2165, 566: 2165, 574: 2165, 576: 2165, 630: 2165, 2165, 2165, 2165}, + {497: 3323, 562: 3325, 3324, 805: 4338}, + {497: 3323, 562: 3325, 3324, 805: 4337}, + {2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 2163, 50: 2163, 461: 2163, 465: 2163, 2163, 2163, 2163, 473: 2163, 480: 2163, 2163, 566: 2163, 574: 2163, 576: 2163, 630: 2163, 2163, 2163, 2163}, + {2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 2164, 50: 2164, 461: 2164, 465: 2164, 2164, 2164, 2164, 473: 2164, 480: 2164, 2164, 566: 2164, 574: 2164, 576: 2164, 630: 2164, 2164, 2164, 2164}, // 1880 - {50: 4339, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2389, 2389, 2389, 2389, 7: 2389, 480: 2389}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4341}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4344}, - {573: 4343}, + {50: 4340, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2390, 2390, 2390, 2390, 7: 2390, 480: 2390}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4342}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4345}, + {573: 4344}, // 1885 {2: 1816, 1816, 1816, 1816, 1816, 8: 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 51: 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 1816, 464: 1816, 558: 1816}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4346, 834: 4345}, - {2344, 2344, 2344, 2344, 7: 2344, 4603, 4604, 480: 2344, 916: 4602}, - {10: 4348, 105: 4396, 109: 4397, 171: 4407, 4406, 4372, 175: 4387, 189: 4409, 214: 4408, 220: 4369, 298: 4376, 4368, 319: 4385, 345: 4392, 4391, 349: 4395, 383: 4403, 494: 4390, 501: 4386, 532: 4381, 637: 4389, 667: 4394, 4393, 670: 4370, 4375, 4373, 4366, 4360, 4374, 678: 4382, 680: 4367, 4399, 4361, 4362, 4363, 4364, 4365, 4388, 4401, 4405, 4400, 4359, 4404, 4371, 695: 4358, 4398, 4357, 4402, 888: 4377, 1136: 4379, 1158: 4356, 4383, 4353, 1178: 4351, 1192: 4354, 1194: 4355, 1213: 4352, 1230: 4378, 4349, 4380, 1288: 4350, 1300: 4384, 1303: 4347, 1328: 4410}, - {2205, 2205, 2205, 2205, 4490, 4496, 4484, 2205, 2205, 2205, 4488, 4497, 4495, 50: 2205, 461: 4489, 465: 3984, 3983, 2212, 4487, 473: 4494, 480: 2205, 4483, 566: 2246, 574: 2335, 576: 4481, 632: 4486, 4479, 4501, 4498, 798: 4482, 820: 4491, 897: 4493, 915: 4499, 924: 4492, 940: 4485, 987: 4500, 4601}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4347, 832: 4346}, + {2345, 2345, 2345, 2345, 7: 2345, 4604, 4605, 480: 2345, 914: 4603}, + {10: 4349, 105: 4397, 109: 4398, 171: 4408, 4407, 4373, 175: 4388, 190: 4410, 214: 4409, 220: 4370, 300: 4377, 4369, 321: 4386, 347: 4393, 4392, 351: 4396, 384: 4404, 493: 4387, 495: 4391, 534: 4382, 635: 4390, 665: 4395, 4394, 668: 4371, 4376, 4374, 4367, 4361, 4375, 676: 4383, 678: 4368, 4400, 4362, 4363, 4364, 4365, 4366, 4389, 4402, 4406, 4401, 4360, 4405, 4372, 693: 4359, 4399, 4358, 4403, 886: 4378, 1135: 4380, 1157: 4357, 4384, 4354, 1177: 4352, 1191: 4355, 1193: 4356, 1212: 4353, 1228: 4379, 4350, 4381, 1286: 4351, 1298: 4385, 1301: 4348, 1326: 4411}, + {2206, 2206, 2206, 2206, 4491, 4497, 4485, 2206, 2206, 2206, 4489, 4498, 4496, 50: 2206, 461: 4490, 465: 3985, 3984, 2213, 4488, 473: 4495, 480: 2206, 4484, 566: 2247, 574: 2336, 576: 4482, 630: 4487, 4480, 4502, 4499, 796: 4483, 818: 4492, 895: 4494, 913: 4500, 922: 4493, 938: 4486, 985: 4501, 4602}, // 1890 - {2205, 2205, 2205, 2205, 4490, 4496, 4484, 2205, 2205, 2205, 4488, 4497, 4495, 50: 2205, 461: 4489, 465: 3984, 3983, 2212, 4487, 473: 4494, 480: 2205, 4483, 566: 2246, 574: 2335, 576: 4481, 632: 4486, 4479, 4501, 4498, 798: 4482, 820: 4491, 897: 4493, 915: 4499, 924: 4492, 940: 4485, 987: 4500, 4480}, - {363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 50: 363, 461: 363, 465: 363, 363, 363, 363, 473: 363, 480: 363, 363, 566: 363, 574: 363, 576: 363, 632: 363, 363, 363, 363}, - {362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 50: 362, 461: 362, 465: 362, 362, 362, 362, 473: 362, 480: 362, 362, 566: 362, 574: 362, 576: 362, 632: 362, 362, 362, 362}, - {361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 50: 361, 461: 361, 465: 361, 361, 361, 361, 473: 361, 480: 361, 361, 566: 361, 574: 361, 576: 361, 632: 361, 361, 361, 361}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 48: 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 729: 278, 731: 278, 750: 3724, 774: 4477}, + {2206, 2206, 2206, 2206, 4491, 4497, 4485, 2206, 2206, 2206, 4489, 4498, 4496, 50: 2206, 461: 4490, 465: 3985, 3984, 2213, 4488, 473: 4495, 480: 2206, 4484, 566: 2247, 574: 2336, 576: 4482, 630: 4487, 4480, 4502, 4499, 796: 4483, 818: 4492, 895: 4494, 913: 4500, 922: 4493, 938: 4486, 985: 4501, 4481}, + {363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 363, 50: 363, 461: 363, 465: 363, 363, 363, 363, 473: 363, 480: 363, 363, 566: 363, 574: 363, 576: 363, 630: 363, 363, 363, 363}, + {362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 362, 50: 362, 461: 362, 465: 362, 362, 362, 362, 473: 362, 480: 362, 362, 566: 362, 574: 362, 576: 362, 630: 362, 362, 362, 362}, + {361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 361, 50: 361, 461: 361, 465: 361, 361, 361, 361, 473: 361, 480: 361, 361, 566: 361, 574: 361, 576: 361, 630: 361, 361, 361, 361}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 48: 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 727: 278, 729: 278, 748: 3725, 772: 4478}, // 1895 - {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 632: 273, 273, 273, 273, 729: 273, 731: 273, 862: 4476}, - {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 48: 271, 50: 271, 461: 271, 3710, 465: 271, 271, 271, 271, 473: 271, 480: 271, 271, 566: 271, 574: 271, 576: 271, 632: 271, 271, 271, 271, 729: 271, 731: 271, 750: 3711, 891: 4474, 896: 3712}, - {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 48: 271, 50: 271, 461: 271, 3710, 465: 271, 271, 271, 271, 473: 271, 480: 271, 271, 566: 271, 574: 271, 576: 271, 632: 271, 271, 271, 271, 729: 271, 731: 271, 750: 3711, 891: 4472, 896: 3712}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4471}, - {355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 48: 355, 50: 355, 461: 355, 355, 465: 355, 355, 355, 355, 473: 355, 480: 355, 355, 566: 355, 574: 355, 576: 355, 632: 355, 355, 355, 355, 729: 355, 731: 355}, + {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 630: 273, 273, 273, 273, 727: 273, 729: 273, 860: 4477}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 48: 271, 50: 271, 461: 271, 3711, 465: 271, 271, 271, 271, 473: 271, 480: 271, 271, 566: 271, 574: 271, 576: 271, 630: 271, 271, 271, 271, 727: 271, 729: 271, 748: 3712, 889: 4475, 894: 3713}, + {271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 271, 48: 271, 50: 271, 461: 271, 3711, 465: 271, 271, 271, 271, 473: 271, 480: 271, 271, 566: 271, 574: 271, 576: 271, 630: 271, 271, 271, 271, 727: 271, 729: 271, 748: 3712, 889: 4473, 894: 3713}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4472}, + {355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 355, 48: 355, 50: 355, 461: 355, 355, 465: 355, 355, 355, 355, 473: 355, 480: 355, 355, 566: 355, 574: 355, 576: 355, 630: 355, 355, 355, 355, 727: 355, 729: 355}, // 1900 - {354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 48: 354, 50: 354, 461: 354, 354, 465: 354, 354, 354, 354, 473: 354, 480: 354, 354, 566: 354, 574: 354, 576: 354, 632: 354, 354, 354, 354, 729: 354, 731: 354}, - {353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 48: 353, 50: 353, 461: 353, 353, 465: 353, 353, 353, 353, 473: 353, 480: 353, 353, 566: 353, 574: 353, 576: 353, 632: 353, 353, 353, 353, 729: 353, 731: 353}, - {352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 48: 352, 50: 352, 461: 352, 352, 465: 352, 352, 352, 352, 473: 352, 480: 352, 352, 566: 352, 574: 352, 576: 352, 632: 352, 352, 352, 352, 729: 352, 731: 352}, - {351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 48: 351, 50: 351, 461: 351, 351, 465: 351, 351, 351, 351, 473: 351, 480: 351, 351, 566: 351, 574: 351, 576: 351, 632: 351, 351, 351, 351, 729: 351, 731: 351}, - {350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 48: 350, 50: 350, 461: 350, 350, 465: 350, 350, 350, 350, 473: 350, 480: 350, 350, 566: 350, 574: 350, 576: 350, 632: 350, 350, 350, 350, 729: 350, 731: 350}, + {354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 354, 48: 354, 50: 354, 461: 354, 354, 465: 354, 354, 354, 354, 473: 354, 480: 354, 354, 566: 354, 574: 354, 576: 354, 630: 354, 354, 354, 354, 727: 354, 729: 354}, + {353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 353, 48: 353, 50: 353, 461: 353, 353, 465: 353, 353, 353, 353, 473: 353, 480: 353, 353, 566: 353, 574: 353, 576: 353, 630: 353, 353, 353, 353, 727: 353, 729: 353}, + {352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 352, 48: 352, 50: 352, 461: 352, 352, 465: 352, 352, 352, 352, 473: 352, 480: 352, 352, 566: 352, 574: 352, 576: 352, 630: 352, 352, 352, 352, 727: 352, 729: 352}, + {351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 351, 48: 351, 50: 351, 461: 351, 351, 465: 351, 351, 351, 351, 473: 351, 480: 351, 351, 566: 351, 574: 351, 576: 351, 630: 351, 351, 351, 351, 727: 351, 729: 351}, + {350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 350, 48: 350, 50: 350, 461: 350, 350, 465: 350, 350, 350, 350, 473: 350, 480: 350, 350, 566: 350, 574: 350, 576: 350, 630: 350, 350, 350, 350, 727: 350, 729: 350}, // 1905 - {349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 48: 349, 50: 349, 461: 349, 349, 465: 349, 349, 349, 349, 473: 349, 480: 349, 349, 566: 349, 574: 349, 576: 349, 632: 349, 349, 349, 349, 729: 349, 731: 349}, - {348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 48: 348, 50: 348, 461: 348, 348, 465: 348, 348, 348, 348, 473: 348, 480: 348, 348, 566: 348, 574: 348, 576: 348, 632: 348, 348, 348, 348, 729: 348, 731: 348}, - {347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 48: 347, 50: 347, 461: 347, 347, 465: 347, 347, 347, 347, 473: 347, 480: 347, 347, 566: 347, 574: 347, 576: 347, 632: 347, 347, 347, 347, 729: 347, 731: 347}, - {346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 48: 346, 50: 346, 461: 346, 346, 465: 346, 346, 346, 346, 473: 346, 480: 346, 346, 566: 346, 574: 346, 576: 346, 632: 346, 346, 346, 346, 729: 346, 731: 346}, - {345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 48: 345, 50: 345, 461: 345, 345, 465: 345, 345, 345, 345, 473: 345, 480: 345, 345, 566: 345, 574: 345, 576: 345, 632: 345, 345, 345, 345, 729: 345, 731: 345}, + {349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 349, 48: 349, 50: 349, 461: 349, 349, 465: 349, 349, 349, 349, 473: 349, 480: 349, 349, 566: 349, 574: 349, 576: 349, 630: 349, 349, 349, 349, 727: 349, 729: 349}, + {348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 348, 48: 348, 50: 348, 461: 348, 348, 465: 348, 348, 348, 348, 473: 348, 480: 348, 348, 566: 348, 574: 348, 576: 348, 630: 348, 348, 348, 348, 727: 348, 729: 348}, + {347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 347, 48: 347, 50: 347, 461: 347, 347, 465: 347, 347, 347, 347, 473: 347, 480: 347, 347, 566: 347, 574: 347, 576: 347, 630: 347, 347, 347, 347, 727: 347, 729: 347}, + {346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 346, 48: 346, 50: 346, 461: 346, 346, 465: 346, 346, 346, 346, 473: 346, 480: 346, 346, 566: 346, 574: 346, 576: 346, 630: 346, 346, 346, 346, 727: 346, 729: 346}, + {345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, 48: 345, 50: 345, 461: 345, 345, 465: 345, 345, 345, 345, 473: 345, 480: 345, 345, 566: 345, 574: 345, 576: 345, 630: 345, 345, 345, 345, 727: 345, 729: 345}, // 1910 - {344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 48: 344, 50: 344, 461: 344, 465: 344, 344, 344, 344, 473: 344, 480: 344, 344, 566: 344, 574: 344, 576: 344, 632: 344, 344, 344, 344, 729: 344, 731: 344}, - {343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 48: 343, 50: 343, 461: 343, 465: 343, 343, 343, 343, 473: 343, 480: 343, 343, 566: 343, 574: 343, 576: 343, 632: 343, 343, 343, 343, 729: 343, 731: 343}, - {339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 48: 339, 50: 339, 461: 339, 339, 465: 339, 339, 339, 339, 473: 339, 480: 339, 339, 566: 339, 574: 339, 576: 339, 632: 339, 339, 339, 339, 729: 339, 731: 339}, - {338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 48: 338, 50: 338, 461: 338, 338, 465: 338, 338, 338, 338, 473: 338, 480: 338, 338, 566: 338, 574: 338, 576: 338, 632: 338, 338, 338, 338, 729: 338, 731: 338}, - {337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 48: 337, 50: 337, 461: 337, 337, 465: 337, 337, 337, 337, 473: 337, 480: 337, 337, 566: 337, 574: 337, 576: 337, 632: 337, 337, 337, 337, 729: 337, 731: 337}, + {344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 344, 48: 344, 50: 344, 461: 344, 465: 344, 344, 344, 344, 473: 344, 480: 344, 344, 566: 344, 574: 344, 576: 344, 630: 344, 344, 344, 344, 727: 344, 729: 344}, + {343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 343, 48: 343, 50: 343, 461: 343, 465: 343, 343, 343, 343, 473: 343, 480: 343, 343, 566: 343, 574: 343, 576: 343, 630: 343, 343, 343, 343, 727: 343, 729: 343}, + {339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 339, 48: 339, 50: 339, 461: 339, 339, 465: 339, 339, 339, 339, 473: 339, 480: 339, 339, 566: 339, 574: 339, 576: 339, 630: 339, 339, 339, 339, 727: 339, 729: 339}, + {338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 338, 48: 338, 50: 338, 461: 338, 338, 465: 338, 338, 338, 338, 473: 338, 480: 338, 338, 566: 338, 574: 338, 576: 338, 630: 338, 338, 338, 338, 727: 338, 729: 338}, + {337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 337, 48: 337, 50: 337, 461: 337, 337, 465: 337, 337, 337, 337, 473: 337, 480: 337, 337, 566: 337, 574: 337, 576: 337, 630: 337, 337, 337, 337, 727: 337, 729: 337}, // 1915 - {336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 48: 336, 50: 336, 461: 336, 336, 465: 336, 336, 336, 336, 473: 336, 480: 336, 336, 566: 336, 574: 336, 576: 336, 632: 336, 336, 336, 336, 729: 336, 731: 336}, - {335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 48: 335, 50: 335, 461: 335, 335, 465: 335, 335, 335, 335, 473: 335, 480: 335, 335, 566: 335, 574: 335, 576: 335, 632: 335, 335, 335, 335, 729: 335, 731: 335}, - {334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 48: 334, 50: 334, 461: 334, 334, 465: 334, 334, 334, 334, 473: 334, 480: 334, 334, 566: 334, 574: 334, 576: 334, 632: 334, 334, 334, 334, 729: 334, 731: 334, 1254: 4470}, - {332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 50: 332, 461: 332, 332, 465: 332, 332, 332, 332, 473: 332, 480: 332, 332, 566: 332, 574: 332, 576: 332, 632: 332, 332, 332, 332}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 3723, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 750: 4467, 761: 3731, 788: 4468}, + {336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 336, 48: 336, 50: 336, 461: 336, 336, 465: 336, 336, 336, 336, 473: 336, 480: 336, 336, 566: 336, 574: 336, 576: 336, 630: 336, 336, 336, 336, 727: 336, 729: 336}, + {335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 335, 48: 335, 50: 335, 461: 335, 335, 465: 335, 335, 335, 335, 473: 335, 480: 335, 335, 566: 335, 574: 335, 576: 335, 630: 335, 335, 335, 335, 727: 335, 729: 335}, + {334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 334, 48: 334, 50: 334, 461: 334, 334, 465: 334, 334, 334, 334, 473: 334, 480: 334, 334, 566: 334, 574: 334, 576: 334, 630: 334, 334, 334, 334, 727: 334, 729: 334, 1252: 4471}, + {332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 332, 50: 332, 461: 332, 332, 465: 332, 332, 332, 332, 473: 332, 480: 332, 332, 566: 332, 574: 332, 576: 332, 630: 332, 332, 332, 332}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 3724, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 748: 4468, 759: 3732, 786: 4469}, // 1920 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 3723, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 750: 4464, 761: 3731, 788: 4465}, - {462: 3723, 750: 4462}, - {462: 3723, 750: 4460}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4459}, - {462: 3723, 750: 4458}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 3724, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 748: 4465, 759: 3732, 786: 4466}, + {462: 3724, 748: 4463}, + {462: 3724, 748: 4461}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4460}, + {462: 3724, 748: 4459}, // 1925 - {323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 50: 323, 461: 323, 465: 323, 323, 323, 323, 473: 323, 480: 323, 323, 566: 323, 574: 323, 576: 323, 632: 323, 323, 323, 323}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 101: 4439, 4441, 104: 4440, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4438, 870: 4457}, - {462: 4453}, - {462: 4443}, - {319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 50: 319, 461: 319, 465: 319, 319, 319, 319, 473: 319, 480: 319, 319, 566: 319, 574: 319, 576: 319, 632: 319, 319, 319, 319}, + {323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 323, 50: 323, 461: 323, 465: 323, 323, 323, 323, 473: 323, 480: 323, 323, 566: 323, 574: 323, 576: 323, 630: 323, 323, 323, 323}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 101: 4440, 4442, 104: 4441, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4439, 868: 4458}, + {462: 4454}, + {462: 4444}, + {319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 319, 50: 319, 461: 319, 465: 319, 319, 319, 319, 473: 319, 480: 319, 319, 566: 319, 574: 319, 576: 319, 630: 319, 319, 319, 319}, // 1930 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 101: 4439, 4441, 104: 4440, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 4436, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 4435, 667: 4394, 4393, 678: 4437, 761: 3731, 788: 4438, 870: 4434, 1136: 4433}, - {316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 14: 316, 50: 316, 461: 316, 316, 465: 316, 316, 316, 316, 473: 316, 480: 316, 316, 494: 316, 532: 316, 566: 316, 574: 316, 576: 316, 632: 316, 316, 316, 316, 637: 316, 831: 4432}, - {315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 14: 315, 50: 315, 461: 315, 315, 465: 315, 315, 315, 315, 473: 315, 480: 315, 315, 494: 315, 532: 315, 566: 315, 574: 315, 576: 315, 632: 315, 315, 315, 315, 637: 315, 831: 4431}, - {314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 14: 314, 50: 314, 461: 314, 314, 465: 314, 314, 314, 314, 473: 314, 480: 314, 314, 494: 314, 532: 314, 566: 314, 574: 314, 576: 314, 632: 314, 314, 314, 314, 637: 314, 667: 4429, 4428, 831: 4430}, - {494: 4423, 637: 4422, 667: 4425, 4424}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 101: 4440, 4442, 104: 4441, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 4437, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 4436, 665: 4395, 4394, 676: 4438, 759: 3732, 786: 4439, 868: 4435, 1135: 4434}, + {316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, 14: 316, 50: 316, 461: 316, 316, 465: 316, 316, 316, 316, 473: 316, 480: 316, 316, 495: 316, 534: 316, 566: 316, 574: 316, 576: 316, 630: 316, 316, 316, 316, 635: 316, 829: 4433}, + {315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 14: 315, 50: 315, 461: 315, 315, 465: 315, 315, 315, 315, 473: 315, 480: 315, 315, 495: 315, 534: 315, 566: 315, 574: 315, 576: 315, 630: 315, 315, 315, 315, 635: 315, 829: 4432}, + {314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 314, 14: 314, 50: 314, 461: 314, 314, 465: 314, 314, 314, 314, 473: 314, 480: 314, 314, 495: 314, 534: 314, 566: 314, 574: 314, 576: 314, 630: 314, 314, 314, 314, 635: 314, 665: 4430, 4429, 829: 4431}, + {495: 4424, 635: 4423, 665: 4426, 4425}, // 1935 - {309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 14: 309, 50: 309, 101: 309, 309, 104: 309, 461: 309, 309, 465: 309, 309, 309, 309, 473: 309, 480: 309, 309, 494: 309, 532: 309, 566: 309, 574: 309, 576: 309, 632: 309, 309, 309, 309, 637: 309}, - {308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 14: 308, 50: 308, 101: 308, 308, 104: 308, 461: 308, 308, 465: 308, 308, 308, 308, 473: 308, 480: 308, 308, 494: 308, 532: 308, 566: 308, 574: 308, 576: 308, 632: 308, 308, 308, 308, 637: 308}, + {309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 309, 14: 309, 50: 309, 101: 309, 309, 104: 309, 461: 309, 309, 465: 309, 309, 309, 309, 473: 309, 480: 309, 309, 495: 309, 534: 309, 566: 309, 574: 309, 576: 309, 630: 309, 309, 309, 309, 635: 309}, + {308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, 14: 308, 50: 308, 101: 308, 308, 104: 308, 461: 308, 308, 465: 308, 308, 308, 308, 473: 308, 480: 308, 308, 495: 308, 534: 308, 566: 308, 574: 308, 576: 308, 630: 308, 308, 308, 308, 635: 308}, {462: 305}, - {299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 48: 299, 50: 299, 461: 299, 299, 465: 299, 299, 299, 299, 473: 299, 480: 299, 299, 566: 299, 574: 299, 576: 299, 632: 299, 299, 299, 299, 729: 299, 731: 299}, - {298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 48: 298, 50: 298, 461: 298, 298, 465: 298, 298, 298, 298, 473: 298, 480: 298, 298, 566: 298, 574: 298, 576: 298, 632: 298, 298, 298, 298, 729: 298, 731: 298}, + {299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, 48: 299, 50: 299, 461: 299, 299, 465: 299, 299, 299, 299, 473: 299, 480: 299, 299, 566: 299, 574: 299, 576: 299, 630: 299, 299, 299, 299, 727: 299, 729: 299}, + {298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 298, 48: 298, 50: 298, 461: 298, 298, 465: 298, 298, 298, 298, 473: 298, 480: 298, 298, 566: 298, 574: 298, 576: 298, 630: 298, 298, 298, 298, 727: 298, 729: 298}, // 1940 - {297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 50: 297, 461: 297, 465: 297, 297, 297, 297, 473: 297, 480: 297, 297, 566: 297, 574: 297, 576: 297, 632: 297, 297, 297, 297}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4421}, - {295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 50: 295, 461: 295, 465: 295, 295, 295, 295, 473: 295, 480: 295, 295, 566: 295, 574: 295, 576: 295, 632: 295, 295, 295, 295}, - {294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 50: 294, 461: 294, 465: 294, 294, 294, 294, 473: 294, 480: 294, 294, 566: 294, 574: 294, 576: 294, 632: 294, 294, 294, 294}, - {292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 14: 292, 50: 292, 101: 292, 292, 104: 292, 461: 292, 465: 292, 292, 292, 292, 473: 292, 480: 292, 292, 494: 292, 532: 292, 566: 292, 574: 292, 576: 292, 632: 292, 292, 292, 292, 637: 292}, + {297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 297, 50: 297, 461: 297, 465: 297, 297, 297, 297, 473: 297, 480: 297, 297, 566: 297, 574: 297, 576: 297, 630: 297, 297, 297, 297}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4422}, + {295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 295, 50: 295, 461: 295, 465: 295, 295, 295, 295, 473: 295, 480: 295, 295, 566: 295, 574: 295, 576: 295, 630: 295, 295, 295, 295}, + {294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 294, 50: 294, 461: 294, 465: 294, 294, 294, 294, 473: 294, 480: 294, 294, 566: 294, 574: 294, 576: 294, 630: 294, 294, 294, 294}, + {292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 292, 14: 292, 50: 292, 101: 292, 292, 104: 292, 461: 292, 465: 292, 292, 292, 292, 473: 292, 480: 292, 292, 495: 292, 534: 292, 566: 292, 574: 292, 576: 292, 630: 292, 292, 292, 292, 635: 292}, // 1945 - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 14: 278, 50: 278, 101: 278, 278, 104: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 494: 278, 532: 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 637: 278, 750: 3724, 774: 4420}, - {290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 14: 290, 50: 290, 101: 290, 290, 104: 290, 461: 290, 465: 290, 290, 290, 290, 473: 290, 480: 290, 290, 494: 290, 532: 290, 566: 290, 574: 290, 576: 290, 632: 290, 290, 290, 290, 637: 290}, - {289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 14: 289, 50: 289, 101: 289, 289, 104: 289, 461: 289, 465: 289, 289, 289, 289, 473: 289, 480: 289, 289, 494: 289, 532: 289, 566: 289, 574: 289, 576: 289, 632: 289, 289, 289, 289, 637: 289}, - {284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 50: 284, 461: 284, 465: 284, 284, 284, 284, 473: 284, 480: 284, 284, 566: 284, 574: 284, 576: 284, 632: 284, 284, 284, 284}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4419}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 14: 278, 50: 278, 101: 278, 278, 104: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 495: 278, 534: 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 635: 278, 748: 3725, 772: 4421}, + {290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 290, 14: 290, 50: 290, 101: 290, 290, 104: 290, 461: 290, 465: 290, 290, 290, 290, 473: 290, 480: 290, 290, 495: 290, 534: 290, 566: 290, 574: 290, 576: 290, 630: 290, 290, 290, 290, 635: 290}, + {289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 289, 14: 289, 50: 289, 101: 289, 289, 104: 289, 461: 289, 465: 289, 289, 289, 289, 473: 289, 480: 289, 289, 495: 289, 534: 289, 566: 289, 574: 289, 576: 289, 630: 289, 289, 289, 289, 635: 289}, + {284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 284, 50: 284, 461: 284, 465: 284, 284, 284, 284, 473: 284, 480: 284, 284, 566: 284, 574: 284, 576: 284, 630: 284, 284, 284, 284}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4420}, // 1950 - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4418}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4417}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 48: 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 729: 278, 731: 278, 750: 3724, 774: 4411}, - {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 632: 273, 273, 273, 273, 729: 273, 731: 273, 862: 4412}, - {280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 48: 4414, 50: 280, 461: 280, 465: 280, 280, 280, 280, 473: 280, 480: 280, 280, 566: 280, 574: 280, 576: 280, 632: 280, 280, 280, 280, 729: 4413, 731: 4415, 861: 4416}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4419}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4418}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 48: 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 727: 278, 729: 278, 748: 3725, 772: 4412}, + {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 630: 273, 273, 273, 273, 727: 273, 729: 273, 860: 4413}, + {280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 280, 48: 4415, 50: 280, 461: 280, 465: 280, 280, 280, 280, 473: 280, 480: 280, 280, 566: 280, 574: 280, 576: 280, 630: 280, 280, 280, 280, 727: 4414, 729: 4416, 859: 4417}, // 1955 - {276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 48: 276, 50: 276, 461: 276, 465: 276, 276, 276, 276, 473: 276, 480: 276, 276, 566: 276, 574: 276, 576: 276, 632: 276, 276, 276, 276, 729: 276, 731: 276}, - {275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 48: 275, 50: 275, 461: 275, 465: 275, 275, 275, 275, 473: 275, 480: 275, 275, 566: 275, 574: 275, 576: 275, 632: 275, 275, 275, 275, 729: 275, 731: 275}, - {274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 48: 274, 50: 274, 461: 274, 465: 274, 274, 274, 274, 473: 274, 480: 274, 274, 566: 274, 574: 274, 576: 274, 632: 274, 274, 274, 274, 729: 274, 731: 274}, - {272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 48: 272, 50: 272, 461: 272, 465: 272, 272, 272, 272, 473: 272, 480: 272, 272, 566: 272, 574: 272, 576: 272, 632: 272, 272, 272, 272, 729: 272, 731: 272}, - {281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 50: 281, 461: 281, 465: 281, 281, 281, 281, 473: 281, 480: 281, 281, 566: 281, 574: 281, 576: 281, 632: 281, 281, 281, 281}, + {276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 276, 48: 276, 50: 276, 461: 276, 465: 276, 276, 276, 276, 473: 276, 480: 276, 276, 566: 276, 574: 276, 576: 276, 630: 276, 276, 276, 276, 727: 276, 729: 276}, + {275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 275, 48: 275, 50: 275, 461: 275, 465: 275, 275, 275, 275, 473: 275, 480: 275, 275, 566: 275, 574: 275, 576: 275, 630: 275, 275, 275, 275, 727: 275, 729: 275}, + {274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 274, 48: 274, 50: 274, 461: 274, 465: 274, 274, 274, 274, 473: 274, 480: 274, 274, 566: 274, 574: 274, 576: 274, 630: 274, 274, 274, 274, 727: 274, 729: 274}, + {272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 272, 48: 272, 50: 272, 461: 272, 465: 272, 272, 272, 272, 473: 272, 480: 272, 272, 566: 272, 574: 272, 576: 272, 630: 272, 272, 272, 272, 727: 272, 729: 272}, + {281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 281, 50: 281, 461: 281, 465: 281, 281, 281, 281, 473: 281, 480: 281, 281, 566: 281, 574: 281, 576: 281, 630: 281, 281, 281, 281}, // 1960 - {282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 50: 282, 461: 282, 465: 282, 282, 282, 282, 473: 282, 480: 282, 282, 566: 282, 574: 282, 576: 282, 632: 282, 282, 282, 282}, - {283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 50: 283, 461: 283, 465: 283, 283, 283, 283, 473: 283, 480: 283, 283, 566: 283, 574: 283, 576: 283, 632: 283, 283, 283, 283}, - {291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 14: 291, 50: 291, 101: 291, 291, 104: 291, 461: 291, 465: 291, 291, 291, 291, 473: 291, 480: 291, 291, 494: 291, 532: 291, 566: 291, 574: 291, 576: 291, 632: 291, 291, 291, 291, 637: 291}, - {296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 50: 296, 461: 296, 465: 296, 296, 296, 296, 473: 296, 480: 296, 296, 566: 296, 574: 296, 576: 296, 632: 296, 296, 296, 296}, - {313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 14: 313, 50: 313, 461: 313, 313, 465: 313, 313, 313, 313, 473: 313, 480: 313, 313, 494: 313, 532: 313, 566: 313, 574: 313, 576: 313, 632: 313, 313, 313, 313, 637: 313, 831: 4427}, + {282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 282, 50: 282, 461: 282, 465: 282, 282, 282, 282, 473: 282, 480: 282, 282, 566: 282, 574: 282, 576: 282, 630: 282, 282, 282, 282}, + {283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 283, 50: 283, 461: 283, 465: 283, 283, 283, 283, 473: 283, 480: 283, 283, 566: 283, 574: 283, 576: 283, 630: 283, 283, 283, 283}, + {291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 291, 14: 291, 50: 291, 101: 291, 291, 104: 291, 461: 291, 465: 291, 291, 291, 291, 473: 291, 480: 291, 291, 495: 291, 534: 291, 566: 291, 574: 291, 576: 291, 630: 291, 291, 291, 291, 635: 291}, + {296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, 50: 296, 461: 296, 465: 296, 296, 296, 296, 473: 296, 480: 296, 296, 566: 296, 574: 296, 576: 296, 630: 296, 296, 296, 296}, + {313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 313, 14: 313, 50: 313, 461: 313, 313, 465: 313, 313, 313, 313, 473: 313, 480: 313, 313, 495: 313, 534: 313, 566: 313, 574: 313, 576: 313, 630: 313, 313, 313, 313, 635: 313, 829: 4428}, // 1965 - {312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 14: 312, 50: 312, 461: 312, 312, 465: 312, 312, 312, 312, 473: 312, 480: 312, 312, 494: 312, 532: 312, 566: 312, 574: 312, 576: 312, 632: 312, 312, 312, 312, 637: 312, 831: 4426}, + {312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 312, 14: 312, 50: 312, 461: 312, 312, 465: 312, 312, 312, 312, 473: 312, 480: 312, 312, 495: 312, 534: 312, 566: 312, 574: 312, 576: 312, 630: 312, 312, 312, 312, 635: 312, 829: 4427}, {462: 307}, {462: 306}, {462: 301}, @@ -8386,886 +8381,886 @@ var ( {462: 304}, {462: 303}, {462: 300}, - {310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 14: 310, 50: 310, 101: 310, 310, 104: 310, 461: 310, 310, 465: 310, 310, 310, 310, 473: 310, 480: 310, 310, 494: 310, 532: 310, 566: 310, 574: 310, 576: 310, 632: 310, 310, 310, 310, 637: 310}, - {311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 14: 311, 50: 311, 101: 311, 311, 104: 311, 461: 311, 311, 465: 311, 311, 311, 311, 473: 311, 480: 311, 311, 494: 311, 532: 311, 566: 311, 574: 311, 576: 311, 632: 311, 311, 311, 311, 637: 311}, + {310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 310, 14: 310, 50: 310, 101: 310, 310, 104: 310, 461: 310, 310, 465: 310, 310, 310, 310, 473: 310, 480: 310, 310, 495: 310, 534: 310, 566: 310, 574: 310, 576: 310, 630: 310, 310, 310, 310, 635: 310}, + {311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 311, 14: 311, 50: 311, 101: 311, 311, 104: 311, 461: 311, 311, 465: 311, 311, 311, 311, 473: 311, 480: 311, 311, 495: 311, 534: 311, 566: 311, 574: 311, 576: 311, 630: 311, 311, 311, 311, 635: 311}, // 1975 - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 101: 4439, 4441, 104: 4440, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4438, 870: 4442}, - {317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 50: 317, 461: 317, 465: 317, 317, 317, 317, 473: 317, 480: 317, 317, 566: 317, 574: 317, 576: 317, 632: 317, 317, 317, 317}, - {501: 3736, 831: 4432}, - {501: 3735, 831: 4431}, - {293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 50: 293, 461: 293, 465: 293, 293, 293, 293, 473: 293, 480: 293, 293, 566: 293, 574: 293, 576: 293, 632: 293, 293, 293, 293}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 101: 4440, 4442, 104: 4441, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4439, 868: 4443}, + {317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, 50: 317, 461: 317, 465: 317, 317, 317, 317, 473: 317, 480: 317, 317, 566: 317, 574: 317, 576: 317, 630: 317, 317, 317, 317}, + {493: 3737, 829: 4433}, + {493: 3736, 829: 4432}, + {293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 293, 50: 293, 461: 293, 465: 293, 293, 293, 293, 473: 293, 480: 293, 293, 566: 293, 574: 293, 576: 293, 630: 293, 293, 293, 293}, // 1980 - {288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 50: 288, 461: 288, 465: 288, 288, 288, 288, 473: 288, 480: 288, 288, 566: 288, 574: 288, 576: 288, 632: 288, 288, 288, 288}, - {287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 50: 287, 461: 287, 465: 287, 287, 287, 287, 473: 287, 480: 287, 287, 566: 287, 574: 287, 576: 287, 632: 287, 287, 287, 287}, - {286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 50: 286, 461: 286, 465: 286, 286, 286, 286, 473: 286, 480: 286, 286, 566: 286, 574: 286, 576: 286, 632: 286, 286, 286, 286}, - {285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 50: 285, 461: 285, 465: 285, 285, 285, 285, 473: 285, 480: 285, 285, 566: 285, 574: 285, 576: 285, 632: 285, 285, 285, 285}, - {318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 50: 318, 461: 318, 465: 318, 318, 318, 318, 473: 318, 480: 318, 318, 566: 318, 574: 318, 576: 318, 632: 318, 318, 318, 318}, + {288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 288, 50: 288, 461: 288, 465: 288, 288, 288, 288, 473: 288, 480: 288, 288, 566: 288, 574: 288, 576: 288, 630: 288, 288, 288, 288}, + {287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 287, 50: 287, 461: 287, 465: 287, 287, 287, 287, 473: 287, 480: 287, 287, 566: 287, 574: 287, 576: 287, 630: 287, 287, 287, 287}, + {286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 286, 50: 286, 461: 286, 465: 286, 286, 286, 286, 473: 286, 480: 286, 286, 566: 286, 574: 286, 576: 286, 630: 286, 286, 286, 286}, + {285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 285, 50: 285, 461: 285, 465: 285, 285, 285, 285, 473: 285, 480: 285, 285, 566: 285, 574: 285, 576: 285, 630: 285, 285, 285, 285}, + {318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 318, 50: 318, 461: 318, 465: 318, 318, 318, 318, 473: 318, 480: 318, 318, 566: 318, 574: 318, 576: 318, 630: 318, 318, 318, 318}, // 1985 - {464: 4445, 565: 4446, 569: 4447, 954: 4448, 1130: 4444}, - {7: 4450, 50: 4449}, + {464: 4446, 565: 4447, 569: 4448, 952: 4449, 1129: 4445}, + {7: 4451, 50: 4450}, {7: 253, 50: 253}, {7: 252, 50: 252}, {7: 251, 50: 251}, // 1990 {7: 250, 50: 250}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 101: 4439, 4441, 104: 4440, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4438, 870: 4452}, - {464: 4445, 565: 4446, 569: 4447, 954: 4451}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 101: 4440, 4442, 104: 4441, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4439, 868: 4453}, + {464: 4446, 565: 4447, 569: 4448, 952: 4452}, {7: 249, 50: 249}, - {320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 50: 320, 461: 320, 465: 320, 320, 320, 320, 473: 320, 480: 320, 320, 566: 320, 574: 320, 576: 320, 632: 320, 320, 320, 320}, + {320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 320, 50: 320, 461: 320, 465: 320, 320, 320, 320, 473: 320, 480: 320, 320, 566: 320, 574: 320, 576: 320, 630: 320, 320, 320, 320}, // 1995 - {464: 4445, 565: 4446, 569: 4447, 954: 4448, 1130: 4454}, - {7: 4450, 50: 4455}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 101: 4439, 4441, 104: 4440, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4438, 870: 4456}, - {321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 50: 321, 461: 321, 465: 321, 321, 321, 321, 473: 321, 480: 321, 321, 566: 321, 574: 321, 576: 321, 632: 321, 321, 321, 321}, - {322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 50: 322, 461: 322, 465: 322, 322, 322, 322, 473: 322, 480: 322, 322, 566: 322, 574: 322, 576: 322, 632: 322, 322, 322, 322}, + {464: 4446, 565: 4447, 569: 4448, 952: 4449, 1129: 4455}, + {7: 4451, 50: 4456}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 101: 4440, 4442, 104: 4441, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4439, 868: 4457}, + {321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 321, 50: 321, 461: 321, 465: 321, 321, 321, 321, 473: 321, 480: 321, 321, 566: 321, 574: 321, 576: 321, 630: 321, 321, 321, 321}, + {322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 322, 50: 322, 461: 322, 465: 322, 322, 322, 322, 473: 322, 480: 322, 322, 566: 322, 574: 322, 576: 322, 630: 322, 322, 322, 322}, // 2000 - {324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 50: 324, 461: 324, 465: 324, 324, 324, 324, 473: 324, 480: 324, 324, 566: 324, 574: 324, 576: 324, 632: 324, 324, 324, 324}, - {325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 50: 325, 461: 325, 465: 325, 325, 325, 325, 473: 325, 480: 325, 325, 566: 325, 574: 325, 576: 325, 632: 325, 325, 325, 325}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4461}, - {326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 50: 326, 461: 326, 465: 326, 326, 326, 326, 473: 326, 480: 326, 326, 566: 326, 574: 326, 576: 326, 632: 326, 326, 326, 326}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4463}, + {324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 324, 50: 324, 461: 324, 465: 324, 324, 324, 324, 473: 324, 480: 324, 324, 566: 324, 574: 324, 576: 324, 630: 324, 324, 324, 324}, + {325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 325, 50: 325, 461: 325, 465: 325, 325, 325, 325, 473: 325, 480: 325, 325, 566: 325, 574: 325, 576: 325, 630: 325, 325, 325, 325}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4462}, + {326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 326, 50: 326, 461: 326, 465: 326, 326, 326, 326, 473: 326, 480: 326, 326, 566: 326, 574: 326, 576: 326, 630: 326, 326, 326, 326}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4464}, // 2005 - {327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 50: 327, 461: 327, 465: 327, 327, 327, 327, 473: 327, 480: 327, 327, 566: 327, 574: 327, 576: 327, 632: 327, 327, 327, 327}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4466}, - {328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 50: 328, 461: 328, 465: 328, 328, 328, 328, 473: 328, 480: 328, 328, 566: 328, 574: 328, 576: 328, 632: 328, 328, 328, 328}, - {329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 50: 329, 461: 329, 465: 329, 329, 329, 329, 473: 329, 480: 329, 329, 566: 329, 574: 329, 576: 329, 632: 329, 329, 329, 329}, - {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3733, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 494: 3734, 532: 3730, 566: 265, 574: 265, 576: 265, 632: 265, 265, 265, 265, 637: 3732, 761: 3731, 788: 4469}, + {327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 327, 50: 327, 461: 327, 465: 327, 327, 327, 327, 473: 327, 480: 327, 327, 566: 327, 574: 327, 576: 327, 630: 327, 327, 327, 327}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4467}, + {328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 328, 50: 328, 461: 328, 465: 328, 328, 328, 328, 473: 328, 480: 328, 328, 566: 328, 574: 328, 576: 328, 630: 328, 328, 328, 328}, + {329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 329, 50: 329, 461: 329, 465: 329, 329, 329, 329, 473: 329, 480: 329, 329, 566: 329, 574: 329, 576: 329, 630: 329, 329, 329, 329}, + {265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 265, 14: 3734, 50: 265, 461: 265, 465: 265, 265, 265, 265, 473: 265, 480: 265, 265, 495: 3735, 534: 3731, 566: 265, 574: 265, 576: 265, 630: 265, 265, 265, 265, 635: 3733, 759: 3732, 786: 4470}, // 2010 - {330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 50: 330, 461: 330, 465: 330, 330, 330, 330, 473: 330, 480: 330, 330, 566: 330, 574: 330, 576: 330, 632: 330, 330, 330, 330}, - {331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 50: 331, 461: 331, 465: 331, 331, 331, 331, 473: 331, 480: 331, 331, 566: 331, 574: 331, 576: 331, 632: 331, 331, 331, 331}, - {333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 48: 333, 50: 333, 461: 333, 333, 465: 333, 333, 333, 333, 473: 333, 480: 333, 333, 566: 333, 574: 333, 576: 333, 632: 333, 333, 333, 333, 729: 333, 731: 333}, - {356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 50: 356, 461: 356, 465: 356, 356, 356, 356, 473: 356, 480: 356, 356, 566: 356, 574: 356, 576: 356, 632: 356, 356, 356, 356}, - {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 632: 273, 273, 273, 273, 729: 273, 731: 273, 862: 4473}, + {330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 330, 50: 330, 461: 330, 465: 330, 330, 330, 330, 473: 330, 480: 330, 330, 566: 330, 574: 330, 576: 330, 630: 330, 330, 330, 330}, + {331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 331, 50: 331, 461: 331, 465: 331, 331, 331, 331, 473: 331, 480: 331, 331, 566: 331, 574: 331, 576: 331, 630: 331, 331, 331, 331}, + {333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 333, 48: 333, 50: 333, 461: 333, 333, 465: 333, 333, 333, 333, 473: 333, 480: 333, 333, 566: 333, 574: 333, 576: 333, 630: 333, 333, 333, 333, 727: 333, 729: 333}, + {356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 356, 50: 356, 461: 356, 465: 356, 356, 356, 356, 473: 356, 480: 356, 356, 566: 356, 574: 356, 576: 356, 630: 356, 356, 356, 356}, + {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 630: 273, 273, 273, 273, 727: 273, 729: 273, 860: 4474}, // 2015 - {357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 48: 4414, 50: 357, 461: 357, 465: 357, 357, 357, 357, 473: 357, 480: 357, 357, 566: 357, 574: 357, 576: 357, 632: 357, 357, 357, 357, 729: 4413, 731: 4415, 861: 4416}, - {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 632: 273, 273, 273, 273, 729: 273, 731: 273, 862: 4475}, - {358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 48: 4414, 50: 358, 461: 358, 465: 358, 358, 358, 358, 473: 358, 480: 358, 358, 566: 358, 574: 358, 576: 358, 632: 358, 358, 358, 358, 729: 4413, 731: 4415, 861: 4416}, - {359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 48: 4414, 50: 359, 461: 359, 465: 359, 359, 359, 359, 473: 359, 480: 359, 359, 566: 359, 574: 359, 576: 359, 632: 359, 359, 359, 359, 729: 4413, 731: 4415, 861: 4416}, - {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 632: 273, 273, 273, 273, 729: 273, 731: 273, 862: 4478}, + {357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 357, 48: 4415, 50: 357, 461: 357, 465: 357, 357, 357, 357, 473: 357, 480: 357, 357, 566: 357, 574: 357, 576: 357, 630: 357, 357, 357, 357, 727: 4414, 729: 4416, 859: 4417}, + {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 630: 273, 273, 273, 273, 727: 273, 729: 273, 860: 4476}, + {358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 358, 48: 4415, 50: 358, 461: 358, 465: 358, 358, 358, 358, 473: 358, 480: 358, 358, 566: 358, 574: 358, 576: 358, 630: 358, 358, 358, 358, 727: 4414, 729: 4416, 859: 4417}, + {359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 359, 48: 4415, 50: 359, 461: 359, 465: 359, 359, 359, 359, 473: 359, 480: 359, 359, 566: 359, 574: 359, 576: 359, 630: 359, 359, 359, 359, 727: 4414, 729: 4416, 859: 4417}, + {273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, 48: 273, 50: 273, 461: 273, 465: 273, 273, 273, 273, 473: 273, 480: 273, 273, 566: 273, 574: 273, 576: 273, 630: 273, 273, 273, 273, 727: 273, 729: 273, 860: 4479}, // 2020 - {360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 48: 4414, 50: 360, 461: 360, 465: 360, 360, 360, 360, 473: 360, 480: 360, 360, 566: 360, 574: 360, 576: 360, 632: 360, 360, 360, 360, 729: 4413, 731: 4415, 861: 4416}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 566: 2334, 574: 2334, 576: 2334, 632: 2334, 639: 2334, 653: 4600, 2674, 2675, 2673, 665: 2334, 2334, 1123: 4599}, - {2268, 2268, 2268, 2268, 7: 2268, 2268, 2268, 50: 2268, 480: 2268}, - {566: 2245}, - {481: 4598}, + {360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, 48: 4415, 50: 360, 461: 360, 465: 360, 360, 360, 360, 473: 360, 480: 360, 360, 566: 360, 574: 360, 576: 360, 630: 360, 360, 360, 360, 727: 4414, 729: 4416, 859: 4417}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 566: 2335, 574: 2335, 576: 2335, 630: 2335, 637: 2335, 651: 4601, 2675, 2676, 2674, 663: 2335, 2335, 1122: 4600}, + {2269, 2269, 2269, 2269, 7: 2269, 2269, 2269, 50: 2269, 480: 2269}, + {566: 2246}, + {481: 4599}, // 2025 - {2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 50: 2235, 461: 2235, 465: 2235, 2235, 2235, 2235, 473: 2235, 480: 2235, 2235, 566: 2235, 574: 2235, 576: 2235, 632: 2235, 2235, 2235, 2235}, - {2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 50: 2234, 461: 2234, 465: 2234, 2234, 2234, 2234, 473: 2234, 480: 2234, 2234, 566: 2234, 574: 2234, 576: 2234, 632: 2234, 2234, 2234, 2234}, - {566: 4594}, - {2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 50: 2231, 461: 2231, 465: 2231, 2231, 2231, 2231, 473: 2231, 480: 2231, 2231, 566: 4593, 574: 2231, 576: 2231, 632: 2231, 2231, 2231, 2231}, - {257: 4591, 347: 4592, 464: 3140, 474: 4335, 4334, 481: 3131, 496: 3135, 560: 3130, 3132, 3134, 3133, 565: 3138, 569: 3139, 578: 4580, 4577, 4578, 4579, 3137, 702: 4333, 3136, 4590, 1060: 4575, 4576, 4588, 1113: 4589, 1180: 4587}, + {2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 50: 2236, 461: 2236, 465: 2236, 2236, 2236, 2236, 473: 2236, 480: 2236, 2236, 566: 2236, 574: 2236, 576: 2236, 630: 2236, 2236, 2236, 2236}, + {2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 2235, 50: 2235, 461: 2235, 465: 2235, 2235, 2235, 2235, 473: 2235, 480: 2235, 2235, 566: 2235, 574: 2235, 576: 2235, 630: 2235, 2235, 2235, 2235}, + {566: 4595}, + {2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 50: 2232, 461: 2232, 465: 2232, 2232, 2232, 2232, 473: 2232, 480: 2232, 2232, 566: 4594, 574: 2232, 576: 2232, 630: 2232, 2232, 2232, 2232}, + {257: 4592, 349: 4593, 464: 3141, 474: 4336, 4335, 481: 3132, 497: 3136, 560: 3131, 3133, 3135, 3134, 565: 3139, 569: 3140, 578: 4581, 4578, 4579, 4580, 3138, 700: 4334, 3137, 4591, 1059: 4576, 4577, 4589, 1112: 4590, 1179: 4588}, // 2030 - {468: 4585}, - {644: 4573}, - {464: 4572}, - {574: 4563}, - {467: 4556}, + {468: 4586}, + {642: 4574}, + {464: 4573}, + {574: 4564}, + {467: 4557}, // 2035 - {2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 50: 2223, 461: 2223, 465: 2223, 2223, 2223, 2223, 473: 2223, 480: 2223, 2223, 566: 2223, 574: 2223, 576: 2223, 632: 2223, 2223, 2223, 2223}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3388, 653: 3390, 2674, 2675, 2673, 728: 3387, 858: 4555}, - {173: 4553, 195: 4554, 468: 4552, 1165: 4551}, - {177: 4550, 237: 4549, 468: 4548, 1284: 4547}, - {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3723, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 632: 278, 278, 278, 278, 750: 3724, 774: 4546}, + {2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 50: 2224, 461: 2224, 465: 2224, 2224, 2224, 2224, 473: 2224, 480: 2224, 2224, 566: 2224, 574: 2224, 576: 2224, 630: 2224, 2224, 2224, 2224}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3389, 651: 3391, 2675, 2676, 2674, 726: 3388, 856: 4556}, + {173: 4554, 196: 4555, 468: 4553, 1164: 4552}, + {178: 4551, 237: 4550, 468: 4549, 1282: 4548}, + {278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 278, 50: 278, 461: 278, 3724, 465: 278, 278, 278, 278, 473: 278, 480: 278, 278, 566: 278, 574: 278, 576: 278, 630: 278, 278, 278, 278, 748: 3725, 772: 4547}, // 2040 - {295: 4545}, - {2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 50: 2207, 461: 2207, 465: 2207, 2207, 2207, 2207, 473: 2207, 480: 2207, 2207, 566: 2207, 574: 2207, 576: 2207, 632: 2207, 2207, 2207, 2207}, - {2204, 2204, 2204, 2204, 4490, 4496, 4484, 2204, 2204, 2204, 4488, 4497, 4495, 50: 2204, 461: 4489, 465: 3984, 3983, 2212, 4487, 473: 4494, 480: 2204, 4483, 566: 2246, 574: 2335, 576: 4481, 632: 4486, 4479, 4501, 4498, 798: 4482, 820: 4491, 897: 4493, 915: 4544, 924: 4492, 940: 4485}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4502}, - {2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 50: 2152, 461: 2152, 4504, 465: 2152, 2152, 2152, 2152, 473: 2152, 480: 2152, 2152, 566: 2152, 574: 2152, 576: 2152, 632: 2152, 2152, 2152, 2152, 638: 2152, 1209: 4503}, + {297: 4546}, + {2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 50: 2208, 461: 2208, 465: 2208, 2208, 2208, 2208, 473: 2208, 480: 2208, 2208, 566: 2208, 574: 2208, 576: 2208, 630: 2208, 2208, 2208, 2208}, + {2205, 2205, 2205, 2205, 4491, 4497, 4485, 2205, 2205, 2205, 4489, 4498, 4496, 50: 2205, 461: 4490, 465: 3985, 3984, 2213, 4488, 473: 4495, 480: 2205, 4484, 566: 2247, 574: 2336, 576: 4482, 630: 4487, 4480, 4502, 4499, 796: 4483, 818: 4492, 895: 4494, 913: 4545, 922: 4493, 938: 4486}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4503}, + {2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 2153, 50: 2153, 461: 2153, 4505, 465: 2153, 2153, 2153, 2153, 473: 2153, 480: 2153, 2153, 566: 2153, 574: 2153, 576: 2153, 630: 2153, 2153, 2153, 2153, 636: 2153, 1208: 4504}, // 2045 - {2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 50: 2194, 461: 2194, 465: 2194, 2194, 2194, 2194, 473: 2194, 480: 2194, 2194, 566: 2194, 574: 2194, 576: 2194, 632: 2194, 2194, 2194, 2194, 638: 4519, 1226: 4520, 4521}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4505}, - {7: 4517, 50: 4516}, - {7: 2150, 50: 2150}, - {7: 278, 50: 278, 462: 3723, 518: 278, 278, 750: 3724, 774: 4514}, + {2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 50: 2195, 461: 2195, 465: 2195, 2195, 2195, 2195, 473: 2195, 480: 2195, 2195, 566: 2195, 574: 2195, 576: 2195, 630: 2195, 2195, 2195, 2195, 636: 4520, 1224: 4521, 4522}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4506}, + {7: 4518, 50: 4517}, + {7: 2151, 50: 2151}, + {7: 278, 50: 278, 462: 3724, 518: 278, 278, 748: 3725, 772: 4515}, // 2050 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4509}, - {50: 4510, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {7: 1268, 50: 1268, 518: 4513, 4512, 933: 4511}, - {7: 2147, 50: 2147}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4510}, + {50: 4511, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {7: 1268, 50: 1268, 518: 4514, 4513, 931: 4512}, + {7: 2148, 50: 2148}, {1267, 1267, 1267, 1267, 7: 1267, 50: 1267, 480: 1267}, // 2055 {1266, 1266, 1266, 1266, 7: 1266, 50: 1266, 480: 1266}, - {7: 1268, 50: 1268, 518: 4513, 4512, 933: 4515}, - {7: 2148, 50: 2148}, - {2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 2151, 50: 2151, 461: 2151, 465: 2151, 2151, 2151, 2151, 473: 2151, 480: 2151, 2151, 566: 2151, 574: 2151, 576: 2151, 632: 2151, 2151, 2151, 2151, 638: 2151}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4518}, - // 2060 + {7: 1268, 50: 1268, 518: 4514, 4513, 931: 4516}, {7: 2149, 50: 2149}, - {199: 4541, 354: 4542, 371: 4543}, - {2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 50: 2193, 461: 2193, 465: 2193, 2193, 2193, 2193, 473: 2193, 480: 2193, 2193, 566: 2193, 574: 2193, 576: 2193, 632: 2193, 2193, 2193, 2193}, - {2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 50: 2189, 461: 4523, 465: 2189, 2189, 2189, 2189, 473: 2189, 480: 2189, 2189, 566: 2189, 574: 2189, 576: 2189, 632: 2189, 2189, 2189, 2189, 1068: 4524, 4525, 1233: 4522}, - {2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 50: 2192, 461: 2192, 465: 2192, 2192, 2192, 2192, 473: 2192, 480: 2192, 2192, 566: 2192, 574: 2192, 576: 2192, 632: 2192, 2192, 2192, 2192}, + {2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 2152, 50: 2152, 461: 2152, 465: 2152, 2152, 2152, 2152, 473: 2152, 480: 2152, 2152, 566: 2152, 574: 2152, 576: 2152, 630: 2152, 2152, 2152, 2152, 636: 2152}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4519}, + // 2060 + {7: 2150, 50: 2150}, + {200: 4542, 356: 4543, 372: 4544}, + {2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 2194, 50: 2194, 461: 2194, 465: 2194, 2194, 2194, 2194, 473: 2194, 480: 2194, 2194, 566: 2194, 574: 2194, 576: 2194, 630: 2194, 2194, 2194, 2194}, + {2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 50: 2190, 461: 4524, 465: 2190, 2190, 2190, 2190, 473: 2190, 480: 2190, 2190, 566: 2190, 574: 2190, 576: 2190, 630: 2190, 2190, 2190, 2190, 1067: 4525, 4526, 1231: 4523}, + {2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 2193, 50: 2193, 461: 2193, 465: 2193, 2193, 2193, 2193, 473: 2193, 480: 2193, 2193, 566: 2193, 574: 2193, 576: 2193, 630: 2193, 2193, 2193, 2193}, // 2065 - {644: 4539, 732: 4528}, - {2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 50: 2188, 461: 4537, 465: 2188, 2188, 2188, 2188, 473: 2188, 480: 2188, 2188, 566: 2188, 574: 2188, 576: 2188, 632: 2188, 2188, 2188, 2188, 1069: 4538}, - {2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 50: 2187, 461: 4526, 465: 2187, 2187, 2187, 2187, 473: 2187, 480: 2187, 2187, 566: 2187, 574: 2187, 576: 2187, 632: 2187, 2187, 2187, 2187, 1068: 4527}, - {732: 4528}, - {2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 50: 2185, 461: 2185, 465: 2185, 2185, 2185, 2185, 473: 2185, 480: 2185, 2185, 566: 2185, 574: 2185, 576: 2185, 632: 2185, 2185, 2185, 2185}, + {642: 4540, 730: 4529}, + {2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 2189, 50: 2189, 461: 4538, 465: 2189, 2189, 2189, 2189, 473: 2189, 480: 2189, 2189, 566: 2189, 574: 2189, 576: 2189, 630: 2189, 2189, 2189, 2189, 1068: 4539}, + {2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 2188, 50: 2188, 461: 4527, 465: 2188, 2188, 2188, 2188, 473: 2188, 480: 2188, 2188, 566: 2188, 574: 2188, 576: 2188, 630: 2188, 2188, 2188, 2188, 1067: 4528}, + {730: 4529}, + {2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 50: 2186, 461: 2186, 465: 2186, 2186, 2186, 2186, 473: 2186, 480: 2186, 2186, 566: 2186, 574: 2186, 576: 2186, 630: 2186, 2186, 2186, 2186}, // 2070 - {70: 4533, 501: 4532, 660: 4531, 662: 4530, 1091: 4529}, - {2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 50: 2191, 461: 2191, 465: 2191, 2191, 2191, 2191, 473: 2191, 480: 2191, 2191, 566: 2191, 574: 2191, 576: 2191, 632: 2191, 2191, 2191, 2191}, - {2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 50: 2184, 461: 2184, 465: 2184, 2184, 2184, 2184, 473: 2184, 480: 2184, 2184, 566: 2184, 574: 2184, 576: 2184, 632: 2184, 2184, 2184, 2184}, - {2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 50: 2183, 461: 2183, 465: 2183, 2183, 2183, 2183, 473: 2183, 480: 2183, 2183, 566: 2183, 574: 2183, 576: 2183, 632: 2183, 2183, 2183, 2183}, - {468: 4536, 481: 4535}, + {70: 4534, 493: 4533, 658: 4532, 660: 4531, 1090: 4530}, + {2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 2192, 50: 2192, 461: 2192, 465: 2192, 2192, 2192, 2192, 473: 2192, 480: 2192, 2192, 566: 2192, 574: 2192, 576: 2192, 630: 2192, 2192, 2192, 2192}, + {2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 2185, 50: 2185, 461: 2185, 465: 2185, 2185, 2185, 2185, 473: 2185, 480: 2185, 2185, 566: 2185, 574: 2185, 576: 2185, 630: 2185, 2185, 2185, 2185}, + {2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 2184, 50: 2184, 461: 2184, 465: 2184, 2184, 2184, 2184, 473: 2184, 480: 2184, 2184, 566: 2184, 574: 2184, 576: 2184, 630: 2184, 2184, 2184, 2184}, + {468: 4537, 481: 4536}, // 2075 - {291: 4534}, - {2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 50: 2181, 461: 2181, 465: 2181, 2181, 2181, 2181, 473: 2181, 480: 2181, 2181, 566: 2181, 574: 2181, 576: 2181, 632: 2181, 2181, 2181, 2181}, - {2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 50: 2182, 461: 2182, 465: 2182, 2182, 2182, 2182, 473: 2182, 480: 2182, 2182, 566: 2182, 574: 2182, 576: 2182, 632: 2182, 2182, 2182, 2182}, - {2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 50: 2180, 461: 2180, 465: 2180, 2180, 2180, 2180, 473: 2180, 480: 2180, 2180, 566: 2180, 574: 2180, 576: 2180, 632: 2180, 2180, 2180, 2180}, - {644: 4539}, + {293: 4535}, + {2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 2182, 50: 2182, 461: 2182, 465: 2182, 2182, 2182, 2182, 473: 2182, 480: 2182, 2182, 566: 2182, 574: 2182, 576: 2182, 630: 2182, 2182, 2182, 2182}, + {2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 2183, 50: 2183, 461: 2183, 465: 2183, 2183, 2183, 2183, 473: 2183, 480: 2183, 2183, 566: 2183, 574: 2183, 576: 2183, 630: 2183, 2183, 2183, 2183}, + {2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 2181, 50: 2181, 461: 2181, 465: 2181, 2181, 2181, 2181, 473: 2181, 480: 2181, 2181, 566: 2181, 574: 2181, 576: 2181, 630: 2181, 2181, 2181, 2181}, + {642: 4540}, // 2080 - {2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 2186, 50: 2186, 461: 2186, 465: 2186, 2186, 2186, 2186, 473: 2186, 480: 2186, 2186, 566: 2186, 574: 2186, 576: 2186, 632: 2186, 2186, 2186, 2186}, - {70: 4533, 501: 4532, 660: 4531, 662: 4530, 1091: 4540}, - {2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 2190, 50: 2190, 461: 2190, 465: 2190, 2190, 2190, 2190, 473: 2190, 480: 2190, 2190, 566: 2190, 574: 2190, 576: 2190, 632: 2190, 2190, 2190, 2190}, - {2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 50: 2197, 461: 2197, 465: 2197, 2197, 2197, 2197, 473: 2197, 480: 2197, 2197, 566: 2197, 574: 2197, 576: 2197, 632: 2197, 2197, 2197, 2197}, - {2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 50: 2196, 461: 2196, 465: 2196, 2196, 2196, 2196, 473: 2196, 480: 2196, 2196, 566: 2196, 574: 2196, 576: 2196, 632: 2196, 2196, 2196, 2196}, + {2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 2187, 50: 2187, 461: 2187, 465: 2187, 2187, 2187, 2187, 473: 2187, 480: 2187, 2187, 566: 2187, 574: 2187, 576: 2187, 630: 2187, 2187, 2187, 2187}, + {70: 4534, 493: 4533, 658: 4532, 660: 4531, 1090: 4541}, + {2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 2191, 50: 2191, 461: 2191, 465: 2191, 2191, 2191, 2191, 473: 2191, 480: 2191, 2191, 566: 2191, 574: 2191, 576: 2191, 630: 2191, 2191, 2191, 2191}, + {2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 2198, 50: 2198, 461: 2198, 465: 2198, 2198, 2198, 2198, 473: 2198, 480: 2198, 2198, 566: 2198, 574: 2198, 576: 2198, 630: 2198, 2198, 2198, 2198}, + {2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 2197, 50: 2197, 461: 2197, 465: 2197, 2197, 2197, 2197, 473: 2197, 480: 2197, 2197, 566: 2197, 574: 2197, 576: 2197, 630: 2197, 2197, 2197, 2197}, // 2085 - {2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 2195, 50: 2195, 461: 2195, 465: 2195, 2195, 2195, 2195, 473: 2195, 480: 2195, 2195, 566: 2195, 574: 2195, 576: 2195, 632: 2195, 2195, 2195, 2195}, - {2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 2206, 50: 2206, 461: 2206, 465: 2206, 2206, 2206, 2206, 473: 2206, 480: 2206, 2206, 566: 2206, 574: 2206, 576: 2206, 632: 2206, 2206, 2206, 2206}, - {467: 2211}, - {2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 50: 2219, 461: 2219, 465: 2219, 2219, 2219, 2219, 473: 2219, 480: 2219, 2219, 566: 2219, 574: 2219, 576: 2219, 632: 2219, 2219, 2219, 2219}, - {2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 50: 2220, 461: 2220, 465: 2220, 2220, 2220, 2220, 473: 2220, 480: 2220, 2220, 566: 2220, 574: 2220, 576: 2220, 632: 2220, 2220, 2220, 2220}, + {2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 2196, 50: 2196, 461: 2196, 465: 2196, 2196, 2196, 2196, 473: 2196, 480: 2196, 2196, 566: 2196, 574: 2196, 576: 2196, 630: 2196, 2196, 2196, 2196}, + {2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 2207, 50: 2207, 461: 2207, 465: 2207, 2207, 2207, 2207, 473: 2207, 480: 2207, 2207, 566: 2207, 574: 2207, 576: 2207, 630: 2207, 2207, 2207, 2207}, + {467: 2212}, + {2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 2220, 50: 2220, 461: 2220, 465: 2220, 2220, 2220, 2220, 473: 2220, 480: 2220, 2220, 566: 2220, 574: 2220, 576: 2220, 630: 2220, 2220, 2220, 2220}, + {2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 50: 2221, 461: 2221, 465: 2221, 2221, 2221, 2221, 473: 2221, 480: 2221, 2221, 566: 2221, 574: 2221, 576: 2221, 630: 2221, 2221, 2221, 2221}, // 2090 - {2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 50: 2218, 461: 2218, 465: 2218, 2218, 2218, 2218, 473: 2218, 480: 2218, 2218, 566: 2218, 574: 2218, 576: 2218, 632: 2218, 2218, 2218, 2218}, - {2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 50: 2217, 461: 2217, 465: 2217, 2217, 2217, 2217, 473: 2217, 480: 2217, 2217, 566: 2217, 574: 2217, 576: 2217, 632: 2217, 2217, 2217, 2217}, - {2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 50: 2216, 461: 2216, 465: 2216, 2216, 2216, 2216, 473: 2216, 480: 2216, 2216, 566: 2216, 574: 2216, 576: 2216, 632: 2216, 2216, 2216, 2216}, - {2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 2221, 50: 2221, 461: 2221, 465: 2221, 2221, 2221, 2221, 473: 2221, 480: 2221, 2221, 566: 2221, 574: 2221, 576: 2221, 632: 2221, 2221, 2221, 2221}, - {2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 50: 2215, 461: 2215, 465: 2215, 2215, 2215, 2215, 473: 2215, 480: 2215, 2215, 566: 2215, 574: 2215, 576: 2215, 632: 2215, 2215, 2215, 2215}, + {2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 2219, 50: 2219, 461: 2219, 465: 2219, 2219, 2219, 2219, 473: 2219, 480: 2219, 2219, 566: 2219, 574: 2219, 576: 2219, 630: 2219, 2219, 2219, 2219}, + {2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 2218, 50: 2218, 461: 2218, 465: 2218, 2218, 2218, 2218, 473: 2218, 480: 2218, 2218, 566: 2218, 574: 2218, 576: 2218, 630: 2218, 2218, 2218, 2218}, + {2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 2217, 50: 2217, 461: 2217, 465: 2217, 2217, 2217, 2217, 473: 2217, 480: 2217, 2217, 566: 2217, 574: 2217, 576: 2217, 630: 2217, 2217, 2217, 2217}, + {2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 50: 2222, 461: 2222, 465: 2222, 2222, 2222, 2222, 473: 2222, 480: 2222, 2222, 566: 2222, 574: 2222, 576: 2222, 630: 2222, 2222, 2222, 2222}, + {2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 2216, 50: 2216, 461: 2216, 465: 2216, 2216, 2216, 2216, 473: 2216, 480: 2216, 2216, 566: 2216, 574: 2216, 576: 2216, 630: 2216, 2216, 2216, 2216}, // 2095 - {2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 50: 2214, 461: 2214, 465: 2214, 2214, 2214, 2214, 473: 2214, 480: 2214, 2214, 566: 2214, 574: 2214, 576: 2214, 632: 2214, 2214, 2214, 2214}, - {2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 2213, 50: 2213, 461: 2213, 465: 2213, 2213, 2213, 2213, 473: 2213, 480: 2213, 2213, 566: 2213, 574: 2213, 576: 2213, 632: 2213, 2213, 2213, 2213}, - {2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 2222, 50: 2222, 461: 2222, 465: 2222, 2222, 2222, 2222, 473: 2222, 480: 2222, 2222, 566: 2222, 574: 2222, 576: 2222, 632: 2222, 2222, 2222, 2222}, - {462: 4557}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4558}, + {2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 2215, 50: 2215, 461: 2215, 465: 2215, 2215, 2215, 2215, 473: 2215, 480: 2215, 2215, 566: 2215, 574: 2215, 576: 2215, 630: 2215, 2215, 2215, 2215}, + {2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 2214, 50: 2214, 461: 2214, 465: 2214, 2214, 2214, 2214, 473: 2214, 480: 2214, 2214, 566: 2214, 574: 2214, 576: 2214, 630: 2214, 2214, 2214, 2214}, + {2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 2223, 50: 2223, 461: 2223, 465: 2223, 2223, 2223, 2223, 473: 2223, 480: 2223, 2223, 566: 2223, 574: 2223, 576: 2223, 630: 2223, 2223, 2223, 2223}, + {462: 4558}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4559}, // 2100 - {50: 4559, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 50: 2210, 461: 2210, 465: 2210, 2210, 2210, 2210, 473: 2210, 480: 2210, 2210, 566: 2210, 574: 2210, 576: 2210, 632: 2210, 2210, 2210, 2210, 1285: 4562, 1315: 4561, 4560}, - {2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 2224, 50: 2224, 461: 2224, 465: 2224, 2224, 2224, 2224, 473: 2224, 480: 2224, 2224, 566: 2224, 574: 2224, 576: 2224, 632: 2224, 2224, 2224, 2224}, - {2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 50: 2209, 461: 2209, 465: 2209, 2209, 2209, 2209, 473: 2209, 480: 2209, 2209, 566: 2209, 574: 2209, 576: 2209, 632: 2209, 2209, 2209, 2209}, - {2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 2208, 50: 2208, 461: 2208, 465: 2208, 2208, 2208, 2208, 473: 2208, 480: 2208, 2208, 566: 2208, 574: 2208, 576: 2208, 632: 2208, 2208, 2208, 2208}, + {50: 4560, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 2211, 50: 2211, 461: 2211, 465: 2211, 2211, 2211, 2211, 473: 2211, 480: 2211, 2211, 566: 2211, 574: 2211, 576: 2211, 630: 2211, 2211, 2211, 2211, 1283: 4563, 1313: 4562, 4561}, + {2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 50: 2225, 461: 2225, 465: 2225, 2225, 2225, 2225, 473: 2225, 480: 2225, 2225, 566: 2225, 574: 2225, 576: 2225, 630: 2225, 2225, 2225, 2225}, + {2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 2210, 50: 2210, 461: 2210, 465: 2210, 2210, 2210, 2210, 473: 2210, 480: 2210, 2210, 566: 2210, 574: 2210, 576: 2210, 630: 2210, 2210, 2210, 2210}, + {2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 2209, 50: 2209, 461: 2209, 465: 2209, 2209, 2209, 2209, 473: 2209, 480: 2209, 2209, 566: 2209, 574: 2209, 576: 2209, 630: 2209, 2209, 2209, 2209}, // 2105 - {462: 4564}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4565}, - {50: 4566, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 50: 2240, 147: 4323, 461: 2240, 465: 3984, 3983, 2240, 2240, 473: 2240, 480: 2240, 2240, 566: 2240, 574: 2240, 576: 2240, 632: 2240, 2240, 2240, 2240, 798: 4567, 921: 4568, 1024: 4569, 1183: 4570}, - {147: 4325, 481: 4571}, + {462: 4565}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4566}, + {50: 4567, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 2241, 50: 2241, 147: 4324, 461: 2241, 465: 3985, 3984, 2241, 2241, 473: 2241, 480: 2241, 2241, 566: 2241, 574: 2241, 576: 2241, 630: 2241, 2241, 2241, 2241, 796: 4568, 919: 4569, 1022: 4570, 1182: 4571}, + {147: 4326, 481: 4572}, // 2110 - {2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 50: 2239, 461: 2239, 465: 2239, 2239, 2239, 2239, 473: 2239, 480: 2239, 2239, 566: 2239, 574: 2239, 576: 2239, 632: 2239, 2239, 2239, 2239}, - {2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 50: 2237, 461: 2237, 465: 2237, 2237, 2237, 2237, 473: 2237, 480: 2237, 2237, 566: 2237, 574: 2237, 576: 2237, 632: 2237, 2237, 2237, 2237}, - {2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 2225, 50: 2225, 461: 2225, 465: 2225, 2225, 2225, 2225, 473: 2225, 480: 2225, 2225, 566: 2225, 574: 2225, 576: 2225, 632: 2225, 2225, 2225, 2225}, - {2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 50: 2238, 461: 2238, 465: 2238, 2238, 2238, 2238, 473: 2238, 480: 2238, 2238, 566: 2238, 574: 2238, 576: 2238, 632: 2238, 2238, 2238, 2238}, - {2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 50: 2226, 461: 2226, 465: 2226, 2226, 2226, 2226, 473: 2226, 480: 2226, 2226, 566: 2226, 574: 2226, 576: 2226, 632: 2226, 2226, 2226, 2226}, + {2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 2240, 50: 2240, 461: 2240, 465: 2240, 2240, 2240, 2240, 473: 2240, 480: 2240, 2240, 566: 2240, 574: 2240, 576: 2240, 630: 2240, 2240, 2240, 2240}, + {2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 2238, 50: 2238, 461: 2238, 465: 2238, 2238, 2238, 2238, 473: 2238, 480: 2238, 2238, 566: 2238, 574: 2238, 576: 2238, 630: 2238, 2238, 2238, 2238}, + {2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 2226, 50: 2226, 461: 2226, 465: 2226, 2226, 2226, 2226, 473: 2226, 480: 2226, 2226, 566: 2226, 574: 2226, 576: 2226, 630: 2226, 2226, 2226, 2226}, + {2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 2239, 50: 2239, 461: 2239, 465: 2239, 2239, 2239, 2239, 473: 2239, 480: 2239, 2239, 566: 2239, 574: 2239, 576: 2239, 630: 2239, 2239, 2239, 2239}, + {2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 50: 2227, 461: 2227, 465: 2227, 2227, 2227, 2227, 473: 2227, 480: 2227, 2227, 566: 2227, 574: 2227, 576: 2227, 630: 2227, 2227, 2227, 2227}, // 2115 - {578: 4580, 4577, 4578, 4579, 1060: 4575, 4576, 4574}, - {2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 2227, 50: 2227, 461: 2227, 465: 2227, 2227, 2227, 2227, 473: 2227, 480: 2227, 2227, 566: 2227, 574: 2227, 576: 2227, 632: 2227, 2227, 2227, 2227}, - {2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 50: 2176, 461: 2176, 465: 2176, 2176, 2176, 2176, 473: 2176, 480: 2176, 2176, 566: 2176, 574: 2176, 576: 2176, 632: 2176, 2176, 2176, 2176}, - {462: 4581}, - {2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 50: 2167, 461: 2167, 2171, 465: 2167, 2167, 2167, 2167, 473: 2167, 480: 2167, 2167, 566: 2167, 574: 2167, 576: 2167, 632: 2167, 2167, 2167, 2167}, + {578: 4581, 4578, 4579, 4580, 1059: 4576, 4577, 4575}, + {2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 50: 2228, 461: 2228, 465: 2228, 2228, 2228, 2228, 473: 2228, 480: 2228, 2228, 566: 2228, 574: 2228, 576: 2228, 630: 2228, 2228, 2228, 2228}, + {2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 50: 2177, 461: 2177, 465: 2177, 2177, 2177, 2177, 473: 2177, 480: 2177, 2177, 566: 2177, 574: 2177, 576: 2177, 630: 2177, 2177, 2177, 2177}, + {462: 4582}, + {2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 2168, 50: 2168, 461: 2168, 2172, 465: 2168, 2168, 2168, 2168, 473: 2168, 480: 2168, 2168, 566: 2168, 574: 2168, 576: 2168, 630: 2168, 2168, 2168, 2168}, // 2120 - {2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 50: 2166, 461: 2166, 2170, 465: 2166, 2166, 2166, 2166, 473: 2166, 480: 2166, 2166, 566: 2166, 574: 2166, 576: 2166, 632: 2166, 2166, 2166, 2166}, - {2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 2165, 50: 2165, 461: 2165, 2169, 465: 2165, 2165, 2165, 2165, 473: 2165, 480: 2165, 2165, 566: 2165, 574: 2165, 576: 2165, 632: 2165, 2165, 2165, 2165}, - {462: 2168}, - {50: 4582, 496: 2648, 725: 4583}, - {2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 50: 2175, 461: 2175, 465: 2175, 2175, 2175, 2175, 473: 2175, 480: 2175, 2175, 566: 2175, 574: 2175, 576: 2175, 632: 2175, 2175, 2175, 2175}, + {2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 2167, 50: 2167, 461: 2167, 2171, 465: 2167, 2167, 2167, 2167, 473: 2167, 480: 2167, 2167, 566: 2167, 574: 2167, 576: 2167, 630: 2167, 2167, 2167, 2167}, + {2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 2166, 50: 2166, 461: 2166, 2170, 465: 2166, 2166, 2166, 2166, 473: 2166, 480: 2166, 2166, 566: 2166, 574: 2166, 576: 2166, 630: 2166, 2166, 2166, 2166}, + {462: 2169}, + {50: 4583, 497: 2649, 721: 4584}, + {2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 2176, 50: 2176, 461: 2176, 465: 2176, 2176, 2176, 2176, 473: 2176, 480: 2176, 2176, 566: 2176, 574: 2176, 576: 2176, 630: 2176, 2176, 2176, 2176}, // 2125 - {50: 4584}, - {2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 2174, 50: 2174, 461: 2174, 465: 2174, 2174, 2174, 2174, 473: 2174, 480: 2174, 2174, 566: 2174, 574: 2174, 576: 2174, 632: 2174, 2174, 2174, 2174}, - {152: 4586}, - {2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 2228, 50: 2228, 461: 2228, 465: 2228, 2228, 2228, 2228, 473: 2228, 480: 2228, 2228, 566: 2228, 574: 2228, 576: 2228, 632: 2228, 2228, 2228, 2228}, - {2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 50: 2229, 461: 2229, 465: 2229, 2229, 2229, 2229, 473: 2229, 480: 2229, 2229, 566: 2229, 574: 2229, 576: 2229, 632: 2229, 2229, 2229, 2229}, + {50: 4585}, + {2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 2175, 50: 2175, 461: 2175, 465: 2175, 2175, 2175, 2175, 473: 2175, 480: 2175, 2175, 566: 2175, 574: 2175, 576: 2175, 630: 2175, 2175, 2175, 2175}, + {152: 4587}, + {2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 2229, 50: 2229, 461: 2229, 465: 2229, 2229, 2229, 2229, 473: 2229, 480: 2229, 2229, 566: 2229, 574: 2229, 576: 2229, 630: 2229, 2229, 2229, 2229}, + {2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 50: 2230, 461: 2230, 465: 2230, 2230, 2230, 2230, 473: 2230, 480: 2230, 2230, 566: 2230, 574: 2230, 576: 2230, 630: 2230, 2230, 2230, 2230}, // 2130 - {2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 50: 2179, 461: 2179, 465: 2179, 2179, 2179, 2179, 473: 2179, 480: 2179, 2179, 566: 2179, 574: 2179, 576: 2179, 632: 2179, 2179, 2179, 2179}, - {2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 50: 2178, 461: 2178, 465: 2178, 2178, 2178, 2178, 473: 2178, 480: 2178, 2178, 566: 2178, 574: 2178, 576: 2178, 632: 2178, 2178, 2178, 2178}, - {2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 2177, 50: 2177, 461: 2177, 465: 2177, 2177, 2177, 2177, 473: 2177, 480: 2177, 2177, 566: 2177, 574: 2177, 576: 2177, 632: 2177, 2177, 2177, 2177}, - {152: 4074}, - {462: 4071}, + {2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 2180, 50: 2180, 461: 2180, 465: 2180, 2180, 2180, 2180, 473: 2180, 480: 2180, 2180, 566: 2180, 574: 2180, 576: 2180, 630: 2180, 2180, 2180, 2180}, + {2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 2179, 50: 2179, 461: 2179, 465: 2179, 2179, 2179, 2179, 473: 2179, 480: 2179, 2179, 566: 2179, 574: 2179, 576: 2179, 630: 2179, 2179, 2179, 2179}, + {2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 2178, 50: 2178, 461: 2178, 465: 2178, 2178, 2178, 2178, 473: 2178, 480: 2178, 2178, 566: 2178, 574: 2178, 576: 2178, 630: 2178, 2178, 2178, 2178}, + {152: 4075}, + {462: 4072}, // 2135 - {2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 2230, 50: 2230, 461: 2230, 465: 2230, 2230, 2230, 2230, 473: 2230, 480: 2230, 2230, 566: 2230, 574: 2230, 576: 2230, 632: 2230, 2230, 2230, 2230}, - {2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 50: 2233, 83: 4595, 85: 4596, 461: 2233, 465: 2233, 2233, 2233, 2233, 473: 2233, 480: 2233, 2233, 566: 2233, 574: 2233, 576: 2233, 632: 2233, 2233, 2233, 2233, 855: 4597}, - {2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 17: 2361, 50: 2361, 81: 2361, 2361, 2361, 2361, 2361, 87: 2361, 461: 2361, 463: 2361, 465: 2361, 2361, 2361, 2361, 470: 2361, 473: 2361, 480: 2361, 2361, 485: 2361, 566: 2361, 574: 2361, 576: 2361, 632: 2361, 2361, 2361, 2361}, - {2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 2360, 17: 2360, 50: 2360, 81: 2360, 2360, 2360, 2360, 2360, 87: 2360, 461: 2360, 463: 2360, 465: 2360, 2360, 2360, 2360, 470: 2360, 473: 2360, 480: 2360, 2360, 485: 2360, 566: 2360, 574: 2360, 576: 2360, 632: 2360, 2360, 2360, 2360}, - {2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 2232, 50: 2232, 461: 2232, 465: 2232, 2232, 2232, 2232, 473: 2232, 480: 2232, 2232, 566: 2232, 574: 2232, 576: 2232, 632: 2232, 2232, 2232, 2232}, + {2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 2231, 50: 2231, 461: 2231, 465: 2231, 2231, 2231, 2231, 473: 2231, 480: 2231, 2231, 566: 2231, 574: 2231, 576: 2231, 630: 2231, 2231, 2231, 2231}, + {2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 2234, 50: 2234, 83: 4596, 85: 4597, 461: 2234, 465: 2234, 2234, 2234, 2234, 473: 2234, 480: 2234, 2234, 566: 2234, 574: 2234, 576: 2234, 630: 2234, 2234, 2234, 2234, 853: 4598}, + {2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 2362, 17: 2362, 50: 2362, 81: 2362, 2362, 2362, 2362, 2362, 87: 2362, 461: 2362, 463: 2362, 465: 2362, 2362, 2362, 2362, 470: 2362, 473: 2362, 480: 2362, 2362, 485: 2362, 566: 2362, 574: 2362, 576: 2362, 630: 2362, 2362, 2362, 2362}, + {2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 2361, 17: 2361, 50: 2361, 81: 2361, 2361, 2361, 2361, 2361, 87: 2361, 461: 2361, 463: 2361, 465: 2361, 2361, 2361, 2361, 470: 2361, 473: 2361, 480: 2361, 2361, 485: 2361, 566: 2361, 574: 2361, 576: 2361, 630: 2361, 2361, 2361, 2361}, + {2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 2233, 50: 2233, 461: 2233, 465: 2233, 2233, 2233, 2233, 473: 2233, 480: 2233, 2233, 566: 2233, 574: 2233, 576: 2233, 630: 2233, 2233, 2233, 2233}, // 2140 - {2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 2236, 50: 2236, 461: 2236, 465: 2236, 2236, 2236, 2236, 473: 2236, 480: 2236, 2236, 566: 2236, 574: 2236, 576: 2236, 632: 2236, 2236, 2236, 2236}, - {566: 2333, 574: 2333, 576: 2333, 632: 2333, 639: 2333, 665: 2333, 2333}, - {2332, 2332, 2332, 2332, 7: 2332, 480: 2332, 566: 2332, 574: 2332, 576: 2332, 632: 2332, 639: 2332, 665: 2332, 2332}, - {2269, 2269, 2269, 2269, 7: 2269, 2269, 2269, 50: 2269, 480: 2269}, - {2391, 2391, 2391, 2391, 7: 2391, 480: 2391}, + {2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 2237, 50: 2237, 461: 2237, 465: 2237, 2237, 2237, 2237, 473: 2237, 480: 2237, 2237, 566: 2237, 574: 2237, 576: 2237, 630: 2237, 2237, 2237, 2237}, + {566: 2334, 574: 2334, 576: 2334, 630: 2334, 637: 2334, 663: 2334, 2334}, + {2333, 2333, 2333, 2333, 7: 2333, 480: 2333, 566: 2333, 574: 2333, 576: 2333, 630: 2333, 637: 2333, 663: 2333, 2333}, + {2270, 2270, 2270, 2270, 7: 2270, 2270, 2270, 50: 2270, 480: 2270}, + {2392, 2392, 2392, 2392, 7: 2392, 480: 2392}, // 2145 + {2344, 2344, 2344, 2344, 7: 2344, 480: 2344}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4606}, {2343, 2343, 2343, 2343, 7: 2343, 480: 2343}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4605}, - {2342, 2342, 2342, 2342, 7: 2342, 480: 2342}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4607}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4346, 834: 4608}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4608}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4347, 832: 4609}, // 2150 - {2344, 2344, 2344, 2344, 7: 2344, 4603, 4604, 480: 2344, 916: 4609}, - {2392, 2392, 2392, 2392, 7: 2392, 480: 2392}, + {2345, 2345, 2345, 2345, 7: 2345, 4604, 4605, 480: 2345, 914: 4610}, {2393, 2393, 2393, 2393, 7: 2393, 480: 2393}, {2394, 2394, 2394, 2394, 7: 2394, 480: 2394}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4615, 969: 4614, 1146: 4613}, + {2395, 2395, 2395, 2395, 7: 2395, 480: 2395}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4616, 967: 4615, 1145: 4614}, // 2155 - {2395, 2395, 2395, 2395, 7: 4617, 480: 2395}, + {2396, 2396, 2396, 2396, 7: 4618, 480: 2396}, {1278, 1278, 1278, 1278, 7: 1278, 480: 1278}, - {1268, 1268, 1268, 1268, 7: 1268, 480: 1268, 518: 4513, 4512, 933: 4616}, + {1268, 1268, 1268, 1268, 7: 1268, 480: 1268, 518: 4514, 4513, 931: 4617}, {1276, 1276, 1276, 1276, 7: 1276, 480: 1276}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4615, 969: 4618}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4616, 967: 4619}, // 2160 {1277, 1277, 1277, 1277, 7: 1277, 480: 1277}, - {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 641: 550, 806: 4621, 824: 4620}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4623}, - {549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 51: 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 462: 549, 480: 549, 496: 549, 532: 549, 559: 549, 641: 549}, - {548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 51: 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 462: 548, 480: 548, 496: 548, 532: 548, 559: 548, 641: 548}, + {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4623, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 639: 550, 804: 4622, 822: 4621}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4624}, + {549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 51: 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 549, 462: 549, 480: 549, 497: 549, 534: 549, 559: 549, 639: 549}, + {548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 51: 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 548, 462: 548, 480: 548, 497: 548, 534: 548, 559: 548, 639: 548}, // 2165 - {2398, 2398, 2398, 2398, 7: 2398, 480: 2398}, - {2367, 2367, 2367, 2367, 7: 2367, 18: 2367, 480: 2367}, - {2366, 2366, 2366, 2366, 7: 4627, 18: 2366, 480: 2366}, - {2337, 2337, 2337, 2337, 7: 2337, 18: 2337, 50: 2337, 98: 2337, 159: 2337, 463: 2337, 480: 2337, 484: 2337, 639: 2337, 641: 2337}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4628, 2674, 2675, 2673}, - // 2170 - {2336, 2336, 2336, 2336, 7: 2336, 18: 2336, 50: 2336, 98: 2336, 159: 2336, 463: 2336, 480: 2336, 484: 2336, 639: 2336, 641: 2336}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4631}, {2399, 2399, 2399, 2399, 7: 2399, 480: 2399}, - {18: 4632}, - {2401, 2401, 2401, 2401, 7: 2401, 480: 2401}, - // 2175 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4635}, + {2368, 2368, 2368, 2368, 7: 2368, 18: 2368, 480: 2368}, + {2367, 2367, 2367, 2367, 7: 4628, 18: 2367, 480: 2367}, + {2338, 2338, 2338, 2338, 7: 2338, 18: 2338, 50: 2338, 98: 2338, 159: 2338, 463: 2338, 480: 2338, 484: 2338, 637: 2338, 639: 2338}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4629, 2675, 2676, 2674}, + // 2170 + {2337, 2337, 2337, 2337, 7: 2337, 18: 2337, 50: 2337, 98: 2337, 159: 2337, 463: 2337, 480: 2337, 484: 2337, 637: 2337, 639: 2337}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4632}, {2400, 2400, 2400, 2400, 7: 2400, 480: 2400}, - {18: 4636}, + {18: 4633}, {2402, 2402, 2402, 2402, 7: 2402, 480: 2402}, - {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 641: 550, 806: 4621, 824: 4638}, - // 2180 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4639}, + // 2175 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4636}, + {2401, 2401, 2401, 2401, 7: 2401, 480: 2401}, + {18: 4637}, {2403, 2403, 2403, 2403, 7: 2403, 480: 2403}, - {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 641: 550, 806: 4621, 824: 4641}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4642}, + {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4623, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 639: 550, 804: 4622, 822: 4639}, + // 2180 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4640}, {2404, 2404, 2404, 2404, 7: 2404, 480: 2404}, - // 2185 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4644}, + {2: 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4623, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 639: 550, 804: 4622, 822: 4642}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4643}, {2405, 2405, 2405, 2405, 7: 2405, 480: 2405}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4646, 2674, 2675, 2673}, - {463: 4647}, - {559: 4648}, - // 2190 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4649}, - {2365, 2365, 2365, 2365, 7: 2365, 215: 4653, 463: 4652, 480: 2365, 1326: 4651, 4650}, + // 2185 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4645}, {2406, 2406, 2406, 2406, 7: 2406, 480: 2406}, - {2364, 2364, 2364, 2364, 7: 2364, 480: 2364}, - {190: 4655}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4647, 2675, 2676, 2674}, + {463: 4648}, + {559: 4649}, + // 2190 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4650}, + {2366, 2366, 2366, 2366, 7: 2366, 215: 4654, 463: 4653, 480: 2366, 1324: 4652, 4651}, + {2407, 2407, 2407, 2407, 7: 2407, 480: 2407}, + {2365, 2365, 2365, 2365, 7: 2365, 480: 2365}, + {191: 4656}, // 2195 - {190: 4654}, - {2362, 2362, 2362, 2362, 7: 2362, 480: 2362}, + {191: 4655}, {2363, 2363, 2363, 2363, 7: 2363, 480: 2363}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4674}, - {566: 4673}, + {2364, 2364, 2364, 2364, 7: 2364, 480: 2364}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4675}, + {566: 4674}, // 2200 - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4671}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4669}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4667}, - {566: 4664}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4663, 2674, 2675, 2673}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4672}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4670}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4668}, + {566: 4665}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4664, 2675, 2676, 2674}, // 2205 - {2373, 2373, 2373, 2373, 7: 2373, 480: 2373}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 4665}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4600, 2674, 2675, 2673, 1123: 4666}, - {2396, 2396, 2396, 2396, 7: 2396, 480: 2396}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4668, 2674, 2675, 2673}, - // 2210 + {2374, 2374, 2374, 2374, 7: 2374, 480: 2374}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 4666}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4601, 2675, 2676, 2674, 1122: 4667}, {2397, 2397, 2397, 2397, 7: 2397, 480: 2397}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4670, 2674, 2675, 2673}, - {2407, 2407, 2407, 2407, 7: 2407, 480: 2407}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 4672}, - {2408, 2408, 2408, 2408, 7: 4627, 480: 2408}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4669, 2675, 2676, 2674}, + // 2210 + {2398, 2398, 2398, 2398, 7: 2398, 480: 2398}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4671, 2675, 2676, 2674}, + {2408, 2408, 2408, 2408, 7: 2408, 480: 2408}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 4673}, + {2409, 2409, 2409, 2409, 7: 4628, 480: 2409}, // 2215 - {2409, 2409, 2409, 2409, 7: 2409, 480: 2409}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4675}, - {2004, 2004, 2004, 2004, 7: 2004, 480: 2004, 660: 4678, 662: 4677, 898: 4676}, {2410, 2410, 2410, 2410, 7: 2410, 480: 2410}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4676}, + {2004, 2004, 2004, 2004, 7: 2004, 480: 2004, 658: 4679, 660: 4678, 896: 4677}, + {2411, 2411, 2411, 2411, 7: 2411, 480: 2411}, {2003, 2003, 2003, 2003, 7: 2003, 480: 2003}, // 2220 {2002, 2002, 2002, 2002, 7: 2002, 480: 2002}, - {136: 4622, 496: 550, 806: 4621, 824: 4680}, - {496: 2648, 725: 4681}, - {2411, 2411, 2411, 2411, 7: 2411, 480: 2411}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 641: 4624, 653: 4626, 2674, 2675, 2673, 777: 4625, 819: 4683}, - // 2225 + {136: 4623, 497: 550, 804: 4622, 822: 4681}, + {497: 2649, 721: 4682}, {2412, 2412, 2412, 2412, 7: 2412, 480: 2412}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 555: 4702, 773: 4816}, - {2418, 2418, 2418, 2418, 7: 2418, 480: 2418}, - {1815, 1815, 1815, 1815, 7: 1815, 103: 1815, 136: 1815, 462: 1815, 480: 1815, 555: 4702, 773: 4770, 806: 1815}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 4761}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 639: 4625, 651: 4627, 2675, 2676, 2674, 775: 4626, 817: 4684}, + // 2225 + {2413, 2413, 2413, 2413, 7: 2413, 480: 2413}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 555: 4703, 771: 4817}, + {2419, 2419, 2419, 2419, 7: 2419, 480: 2419}, + {1815, 1815, 1815, 1815, 7: 1815, 103: 1815, 136: 1815, 462: 1815, 480: 1815, 555: 4703, 771: 4771, 804: 1815}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 4762}, // 2230 - {566: 4300, 574: 4694, 576: 4689, 632: 4692, 639: 4301, 665: 4693, 4690, 815: 4691, 1173: 4695}, - {566: 4755}, - {2: 2348, 2348, 2348, 2348, 2348, 8: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 51: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 462: 2348, 566: 4300, 639: 4301, 815: 4711, 1048: 4749}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 470: 1815, 555: 4702, 773: 4743}, - {2: 2348, 2348, 2348, 2348, 2348, 8: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 51: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 462: 2348, 470: 2348, 566: 4300, 639: 4301, 815: 4711, 1048: 4712}, + {566: 4301, 574: 4695, 576: 4690, 630: 4693, 637: 4302, 663: 4694, 4691, 813: 4692, 1172: 4696}, + {566: 4756}, + {2: 2349, 2349, 2349, 2349, 2349, 8: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 51: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 462: 2349, 566: 4301, 637: 4302, 813: 4712, 1046: 4750}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 470: 1815, 555: 4703, 771: 4744}, + {2: 2349, 2349, 2349, 2349, 2349, 8: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 51: 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 2349, 462: 2349, 470: 2349, 566: 4301, 637: 4302, 813: 4712, 1046: 4713}, // 2235 - {566: 4700}, - {462: 4696}, + {566: 4701}, + {462: 4697}, {432, 432, 432, 432, 7: 432, 50: 432, 480: 432}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4697}, - {50: 4698, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4698}, + {50: 4699, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, // 2240 - {2240, 2240, 2240, 2240, 7: 2240, 50: 2240, 147: 4323, 465: 3984, 3983, 480: 2240, 798: 4324, 921: 4568, 1024: 4699}, - {2198, 2198, 2198, 2198, 7: 2198, 50: 2198, 480: 2198}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 555: 4702, 773: 4701}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 1811, 653: 4706, 2674, 2675, 2673, 864: 4705}, - {465: 3984, 3983, 798: 4703}, + {2241, 2241, 2241, 2241, 7: 2241, 50: 2241, 147: 4324, 465: 3985, 3984, 480: 2241, 796: 4325, 919: 4569, 1022: 4700}, + {2199, 2199, 2199, 2199, 7: 2199, 50: 2199, 480: 2199}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 462: 1815, 555: 4703, 771: 4702}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 1811, 651: 4707, 2675, 2676, 2674, 862: 4706}, + {465: 3985, 3984, 796: 4704}, // 2245 - {573: 4704}, - {1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 51: 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 462: 1814, 464: 1814, 470: 1814, 480: 1814, 558: 1814, 806: 1814}, - {462: 4707}, + {573: 4705}, + {1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 51: 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 1814, 462: 1814, 464: 1814, 470: 1814, 480: 1814, 558: 1814, 804: 1814}, + {462: 4708}, {462: 1810}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4708}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4709}, // 2250 - {7: 4517, 50: 4709}, - {634: 4501, 897: 4710}, - {2199, 2199, 2199, 2199, 7: 2199, 50: 2199, 480: 2199}, - {2: 2347, 2347, 2347, 2347, 2347, 8: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 51: 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 2347, 462: 2347, 470: 2347}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 1811, 470: 1811, 653: 4714, 2674, 2675, 2673, 864: 4715, 929: 4713}, + {7: 4518, 50: 4710}, + {632: 4502, 895: 4711}, + {2200, 2200, 2200, 2200, 7: 2200, 50: 2200, 480: 2200}, + {2: 2348, 2348, 2348, 2348, 2348, 8: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 51: 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 2348, 462: 2348, 470: 2348}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 1811, 470: 1811, 651: 4715, 2675, 2676, 2674, 862: 4716, 927: 4714}, // 2255 - {462: 4723}, - {82: 4721, 462: 1810, 470: 1810}, - {462: 1801, 470: 4716}, - {140: 4719, 170: 4718, 183: 4720, 892: 4717}, + {462: 4724}, + {82: 4722, 462: 1810, 470: 1810}, + {462: 1801, 470: 4717}, + {140: 4720, 170: 4719, 184: 4721, 890: 4718}, {462: 1800}, // 2260 {1794, 1794, 1794, 1794, 1794, 7: 1794, 17: 1794, 50: 1794, 81: 1794, 1794, 1794, 1794, 1794, 87: 1794, 461: 1794, 1794, 1794, 470: 1794, 480: 1794, 485: 1794}, {1793, 1793, 1793, 1793, 1793, 7: 1793, 17: 1793, 50: 1793, 81: 1793, 1793, 1793, 1793, 1793, 87: 1793, 461: 1793, 1793, 1793, 470: 1793, 480: 1793, 485: 1793}, {1792, 1792, 1792, 1792, 1792, 7: 1792, 17: 1792, 50: 1792, 81: 1792, 1792, 1792, 1792, 1792, 87: 1792, 461: 1792, 1792, 1792, 470: 1792, 480: 1792, 485: 1792}, - {140: 4719, 170: 4718, 183: 4720, 892: 4722}, + {140: 4720, 170: 4719, 184: 4721, 890: 4723}, {462: 1799}, // 2265 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4724}, - {7: 4517, 50: 4725}, - {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 866: 4726}, - {2200, 2200, 2200, 2200, 4731, 7: 2200, 17: 4728, 50: 2200, 82: 4735, 4595, 4320, 4596, 87: 4319, 463: 4730, 470: 4734, 480: 2200, 843: 4732, 845: 4729, 855: 4733, 865: 4727}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4725}, + {7: 4518, 50: 4726}, + {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 864: 4727}, + {2201, 2201, 2201, 2201, 4732, 7: 2201, 17: 4729, 50: 2201, 82: 4736, 4596, 4321, 4597, 87: 4320, 463: 4731, 470: 4735, 480: 2201, 841: 4733, 843: 4730, 853: 4734, 863: 4728}, {1808, 1808, 1808, 1808, 1808, 7: 1808, 17: 1808, 50: 1808, 81: 1808, 1808, 1808, 1808, 1808, 87: 1808, 463: 1808, 470: 1808, 480: 1808, 485: 1808}, // 2270 - {487: 4172, 496: 1999, 726: 4741}, + {487: 4173, 497: 1999, 724: 4742}, {1806, 1806, 1806, 1806, 1806, 7: 1806, 17: 1806, 50: 1806, 81: 1806, 1806, 1806, 1806, 1806, 87: 1806, 463: 1806, 470: 1806, 480: 1806, 485: 1806}, - {353: 4739}, - {464: 4738}, + {355: 4740}, + {464: 4739}, {1803, 1803, 1803, 1803, 1803, 7: 1803, 17: 1803, 50: 1803, 81: 1803, 1803, 1803, 1803, 1803, 87: 1803, 463: 1803, 470: 1803, 480: 1803, 485: 1803}, // 2275 {1802, 1802, 1802, 1802, 1802, 7: 1802, 17: 1802, 50: 1802, 81: 1802, 1802, 1802, 1802, 1802, 87: 1802, 463: 1802, 470: 1802, 480: 1802, 485: 1802}, - {140: 4719, 170: 4718, 183: 4720, 892: 4737}, - {140: 4719, 170: 4718, 183: 4720, 892: 4736}, + {140: 4720, 170: 4719, 184: 4721, 890: 4738}, + {140: 4720, 170: 4719, 184: 4721, 890: 4737}, {1795, 1795, 1795, 1795, 1795, 7: 1795, 17: 1795, 50: 1795, 81: 1795, 1795, 1795, 1795, 1795, 87: 1795, 461: 1795, 463: 1795, 470: 1795, 480: 1795, 485: 1795}, {1796, 1796, 1796, 1796, 1796, 7: 1796, 17: 1796, 50: 1796, 81: 1796, 1796, 1796, 1796, 1796, 87: 1796, 461: 1796, 463: 1796, 470: 1796, 480: 1796, 485: 1796}, // 2280 {1804, 1804, 1804, 1804, 1804, 7: 1804, 17: 1804, 50: 1804, 81: 1804, 1804, 1804, 1804, 1804, 87: 1804, 463: 1804, 470: 1804, 480: 1804, 485: 1804}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4740, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4741, 2675, 2676, 2674}, {1805, 1805, 1805, 1805, 1805, 7: 1805, 17: 1805, 50: 1805, 81: 1805, 1805, 1805, 1805, 1805, 87: 1805, 463: 1805, 470: 1805, 480: 1805, 485: 1805}, - {496: 2648, 725: 2647, 734: 4742}, + {497: 2649, 721: 2648, 731: 4743}, {1807, 1807, 1807, 1807, 1807, 7: 1807, 17: 1807, 50: 1807, 81: 1807, 1807, 1807, 1807, 1807, 87: 1807, 463: 1807, 470: 1807, 480: 1807, 485: 1807}, // 2285 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 1811, 470: 1811, 653: 4714, 2674, 2675, 2673, 864: 4715, 929: 4744}, - {462: 4745}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4746}, - {7: 4517, 50: 4747}, - {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 866: 4748}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 1811, 470: 1811, 651: 4715, 2675, 2676, 2674, 862: 4716, 927: 4745}, + {462: 4746}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4747}, + {7: 4518, 50: 4748}, + {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 864: 4749}, // 2290 - {2201, 2201, 2201, 2201, 4731, 7: 2201, 17: 4728, 50: 2201, 82: 4735, 4595, 4320, 4596, 87: 4319, 463: 4730, 470: 4734, 480: 2201, 843: 4732, 845: 4729, 855: 4733, 865: 4727}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 1811, 653: 4706, 2674, 2675, 2673, 864: 4750}, - {462: 4751}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4752}, - {7: 4517, 50: 4753}, + {2202, 2202, 2202, 2202, 4732, 7: 2202, 17: 4729, 50: 2202, 82: 4736, 4596, 4321, 4597, 87: 4320, 463: 4731, 470: 4735, 480: 2202, 841: 4733, 843: 4730, 853: 4734, 863: 4728}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 1811, 651: 4707, 2675, 2676, 2674, 862: 4751}, + {462: 4752}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4753}, + {7: 4518, 50: 4754}, // 2295 - {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 866: 4754}, - {2202, 2202, 2202, 2202, 4731, 7: 2202, 17: 4728, 50: 2202, 82: 4735, 4595, 4320, 4596, 87: 4319, 463: 4730, 470: 4734, 480: 2202, 843: 4732, 845: 4729, 855: 4733, 865: 4727}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 1811, 470: 1811, 653: 4714, 2674, 2675, 2673, 864: 4715, 929: 4756}, - {462: 4757}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 4758}, + {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 864: 4755}, + {2203, 2203, 2203, 2203, 4732, 7: 2203, 17: 4729, 50: 2203, 82: 4736, 4596, 4321, 4597, 87: 4320, 463: 4731, 470: 4735, 480: 2203, 841: 4733, 843: 4730, 853: 4734, 863: 4728}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 1811, 470: 1811, 651: 4715, 2675, 2676, 2674, 862: 4716, 927: 4757}, + {462: 4758}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 4759}, // 2300 - {7: 4517, 50: 4759}, - {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 866: 4760}, - {2203, 2203, 2203, 2203, 4731, 7: 2203, 17: 4728, 50: 2203, 82: 4735, 4595, 4320, 4596, 87: 4319, 463: 4730, 470: 4734, 480: 2203, 843: 4732, 845: 4729, 855: 4733, 865: 4727}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4762, 2674, 2675, 2673}, - {222: 4764, 231: 4766, 234: 4765, 1119: 4763}, + {7: 4518, 50: 4760}, + {1809, 1809, 1809, 1809, 1809, 7: 1809, 17: 1809, 50: 1809, 82: 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 480: 1809, 864: 4761}, + {2204, 2204, 2204, 2204, 4732, 7: 2204, 17: 4729, 50: 2204, 82: 4736, 4596, 4321, 4597, 87: 4320, 463: 4731, 470: 4735, 480: 2204, 841: 4733, 843: 4730, 853: 4734, 863: 4728}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4763, 2675, 2676, 2674}, + {222: 4765, 231: 4767, 234: 4766, 1118: 4764}, // 2305 - {462: 4767}, + {462: 4768}, + {50: 2159, 462: 2159}, {50: 2158, 462: 2158}, {50: 2157, 462: 2157}, - {50: 2156, 462: 2156}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 4768}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 4769}, // 2310 - {7: 4041, 50: 4769}, - {2415, 2415, 2415, 2415, 7: 2415, 480: 2415}, - {550, 550, 550, 550, 7: 550, 103: 550, 136: 4622, 462: 550, 480: 550, 806: 4621, 824: 4771}, - {2094, 2094, 2094, 2094, 7: 2094, 103: 4773, 462: 4774, 480: 2094, 1079: 4772}, - {2417, 2417, 2417, 2417, 7: 2417, 480: 2417}, + {7: 4042, 50: 4770}, + {2416, 2416, 2416, 2416, 7: 2416, 480: 2416}, + {550, 550, 550, 550, 7: 550, 103: 550, 136: 4623, 462: 550, 480: 550, 804: 4622, 822: 4772}, + {2094, 2094, 2094, 2094, 7: 2094, 103: 4774, 462: 4775, 480: 2094, 1078: 4773}, + {2418, 2418, 2418, 2418, 7: 2418, 480: 2418}, // 2315 - {496: 2648, 725: 4815}, - {480: 4777, 936: 4776, 1078: 4775}, - {7: 4813, 50: 4812}, + {497: 2649, 721: 4816}, + {480: 4778, 934: 4777, 1077: 4776}, + {7: 4814, 50: 4813}, {7: 2092, 50: 2092}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4778, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4779, 2675, 2676, 2674}, // 2320 - {4: 2071, 2071, 7: 2071, 16: 2071, 18: 2071, 2071, 2071, 22: 2071, 2071, 2071, 2071, 50: 2071, 146: 4783, 330: 4782, 462: 2071, 468: 4781, 491: 4780, 639: 2071, 1248: 4779}, - {4: 2084, 2084, 7: 2084, 16: 2084, 18: 2084, 2084, 2084, 22: 2084, 2084, 2084, 2084, 50: 2084, 462: 2084, 639: 2084, 935: 4799}, - {339: 4784, 533: 4785}, - {4: 2068, 2068, 7: 2068, 16: 2068, 18: 2068, 2068, 2068, 22: 2068, 2068, 2068, 2068, 50: 2068, 462: 2068, 639: 2068}, - {4: 2066, 2066, 7: 2066, 16: 2066, 18: 2066, 2066, 2066, 22: 2066, 2066, 2066, 2066, 50: 2066, 462: 2066, 639: 2066}, + {4: 2071, 2071, 7: 2071, 16: 2071, 18: 2071, 20: 2071, 2071, 2071, 2071, 2071, 2071, 50: 2071, 146: 4784, 332: 4783, 462: 2071, 468: 4782, 491: 4781, 637: 2071, 1246: 4780}, + {4: 2084, 2084, 7: 2084, 16: 2084, 18: 2084, 20: 2084, 2084, 2084, 2084, 2084, 2084, 50: 2084, 462: 2084, 637: 2084, 933: 4800}, + {341: 4785, 532: 4786}, + {4: 2068, 2068, 7: 2068, 16: 2068, 18: 2068, 20: 2068, 2068, 2068, 2068, 2068, 2068, 50: 2068, 462: 2068, 637: 2068}, + {4: 2066, 2066, 7: 2066, 16: 2066, 18: 2066, 20: 2066, 2066, 2066, 2066, 2066, 2066, 50: 2066, 462: 2066, 637: 2066}, // 2325 - {4: 2065, 2065, 7: 2065, 16: 2065, 18: 2065, 2065, 2065, 22: 2065, 2065, 2065, 2065, 50: 2065, 462: 2065, 639: 2065}, - {384: 4794}, - {462: 4786}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 647: 4788, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4789, 932: 4790, 1059: 4787}, - {7: 4792, 50: 4791}, + {4: 2065, 2065, 7: 2065, 16: 2065, 18: 2065, 20: 2065, 2065, 2065, 2065, 2065, 2065, 50: 2065, 462: 2065, 637: 2065}, + {385: 4795}, + {462: 4787}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 645: 4789, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4790, 930: 4791, 1058: 4788}, + {7: 4793, 50: 4792}, // 2330 {7: 1891, 50: 1891}, - {7: 1890, 50: 1890, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, + {7: 1890, 50: 1890, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, {7: 1878, 50: 1878}, - {4: 2067, 2067, 7: 2067, 16: 2067, 18: 2067, 2067, 2067, 22: 2067, 2067, 2067, 2067, 50: 2067, 462: 2067, 639: 2067}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 647: 4788, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4789, 932: 4793}, + {4: 2067, 2067, 7: 2067, 16: 2067, 18: 2067, 20: 2067, 2067, 2067, 2067, 2067, 2067, 50: 2067, 462: 2067, 637: 2067}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 645: 4789, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4790, 930: 4794}, // 2335 {7: 1877, 50: 1877}, - {462: 4796, 647: 4795}, - {4: 2070, 2070, 7: 2070, 16: 2070, 18: 2070, 2070, 2070, 22: 2070, 2070, 2070, 2070, 50: 2070, 462: 2070, 639: 2070}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 647: 4788, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4789, 932: 4790, 1059: 4797}, - {7: 4792, 50: 4798}, + {462: 4797, 645: 4796}, + {4: 2070, 2070, 7: 2070, 16: 2070, 18: 2070, 20: 2070, 2070, 2070, 2070, 2070, 2070, 50: 2070, 462: 2070, 637: 2070}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 645: 4789, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4790, 930: 4791, 1058: 4798}, + {7: 4793, 50: 4799}, // 2340 - {4: 2069, 2069, 7: 2069, 16: 2069, 18: 2069, 2069, 2069, 22: 2069, 2069, 2069, 2069, 50: 2069, 462: 2069, 639: 2069}, - {4: 4132, 4803, 7: 2089, 16: 4090, 18: 4140, 4133, 4136, 22: 4135, 4138, 4139, 4141, 50: 2089, 462: 4801, 639: 4137, 766: 4142, 800: 4802, 1290: 4800}, + {4: 2069, 2069, 7: 2069, 16: 2069, 18: 2069, 20: 2069, 2069, 2069, 2069, 2069, 2069, 50: 2069, 462: 2069, 637: 2069}, + {4: 4133, 4804, 7: 2089, 16: 4091, 18: 4141, 20: 4134, 4137, 4136, 4139, 4140, 4142, 50: 2089, 462: 4802, 637: 4138, 764: 4143, 798: 4803, 1288: 4801}, {7: 2090, 50: 2090}, - {100: 4806, 1121: 4805, 1289: 4804}, - {2083, 2083, 4: 2083, 2083, 7: 2083, 16: 2083, 18: 2083, 2083, 2083, 22: 2083, 2083, 2083, 2083, 50: 2083, 462: 2083, 639: 2083}, + {100: 4807, 1120: 4806, 1287: 4805}, + {2083, 2083, 4: 2083, 2083, 7: 2083, 16: 2083, 18: 2083, 20: 2083, 2083, 2083, 2083, 2083, 2083, 50: 2083, 462: 2083, 637: 2083}, // 2345 - {19: 4271}, - {7: 4810, 50: 4809}, + {20: 4272}, + {7: 4811, 50: 4810}, {7: 2087, 50: 2087}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4807, 2674, 2675, 2673}, - {4: 2084, 2084, 7: 2084, 16: 2084, 18: 2084, 2084, 2084, 22: 2084, 2084, 2084, 2084, 50: 2084, 639: 2084, 935: 4808}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4808, 2675, 2676, 2674}, + {4: 2084, 2084, 7: 2084, 16: 2084, 18: 2084, 20: 2084, 2084, 2084, 2084, 2084, 2084, 50: 2084, 637: 2084, 933: 4809}, // 2350 - {4: 4132, 4803, 7: 2085, 16: 4090, 18: 4140, 4133, 4136, 22: 4135, 4138, 4139, 4141, 50: 2085, 639: 4137, 766: 4142, 800: 4802}, + {4: 4133, 4804, 7: 2085, 16: 4091, 18: 4141, 20: 4134, 4137, 4136, 4139, 4140, 4142, 50: 2085, 637: 4138, 764: 4143, 798: 4803}, {7: 2088, 50: 2088}, - {100: 4806, 1121: 4811}, + {100: 4807, 1120: 4812}, {7: 2086, 50: 2086}, - {2093, 2093, 2093, 2093, 7: 2093, 461: 2093, 2093, 2093, 467: 2093, 479: 2093, 2093, 491: 2093, 495: 2093, 559: 2093, 636: 2093}, + {2093, 2093, 2093, 2093, 7: 2093, 461: 2093, 2093, 2093, 467: 2093, 479: 2093, 2093, 491: 2093, 496: 2093, 559: 2093, 634: 2093}, // 2355 - {480: 4777, 936: 4814}, + {480: 4778, 934: 4815}, {7: 2091, 50: 2091}, - {2416, 2416, 2416, 2416, 7: 2416, 480: 2416}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4818, 653: 4038, 2674, 2675, 2673, 733: 4346, 834: 4817}, - {2344, 2344, 2344, 2344, 7: 2344, 4603, 4604, 480: 2344, 916: 4826}, + {2417, 2417, 2417, 2417, 7: 2417, 480: 2417}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4819, 651: 4039, 2675, 2676, 2674, 732: 4347, 832: 4818}, + {2345, 2345, 2345, 2345, 7: 2345, 4604, 4605, 480: 2345, 914: 4827}, // 2360 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 566: 2335, 574: 2335, 576: 2335, 632: 2335, 4479, 639: 2335, 653: 4038, 2674, 2675, 2673, 665: 2335, 2335, 733: 4346, 820: 4688, 834: 4820, 890: 4821, 952: 4822, 1124: 4819}, - {7: 4824, 50: 4823}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 566: 2336, 574: 2336, 576: 2336, 630: 2336, 4480, 637: 2336, 651: 4039, 2675, 2676, 2674, 663: 2336, 2336, 732: 4347, 818: 4689, 832: 4821, 888: 4822, 950: 4823, 1123: 4820}, + {7: 4825, 50: 4824}, {7: 429, 50: 429}, {7: 428, 50: 428}, {7: 427, 50: 427}, // 2365 - {2419, 2419, 2419, 2419, 7: 2419, 480: 2419}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 566: 2335, 574: 2335, 576: 2335, 632: 2335, 4479, 639: 2335, 653: 4038, 2674, 2675, 2673, 665: 2335, 2335, 733: 4346, 820: 4688, 834: 4820, 890: 4821, 952: 4825}, - {7: 426, 50: 426}, {2420, 2420, 2420, 2420, 7: 2420, 480: 2420}, - {14: 3733, 494: 3734, 637: 3732, 761: 4828}, - // 2370 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 468: 4830, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 4829}, - {257, 257, 257, 257, 7: 257, 473: 4832, 480: 257, 1070: 4834}, - {257, 257, 257, 257, 7: 257, 473: 4832, 480: 257, 1070: 4831}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 566: 2336, 574: 2336, 576: 2336, 630: 2336, 4480, 637: 2336, 651: 4039, 2675, 2676, 2674, 663: 2336, 2336, 732: 4347, 818: 4689, 832: 4821, 888: 4822, 950: 4826}, + {7: 426, 50: 426}, {2421, 2421, 2421, 2421, 7: 2421, 480: 2421}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3388, 653: 3390, 2674, 2675, 2673, 728: 3387, 858: 4833}, + {14: 3734, 495: 3735, 635: 3733, 759: 4829}, + // 2370 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 468: 4831, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 4830}, + {257, 257, 257, 257, 7: 257, 473: 4833, 480: 257, 1069: 4835}, + {257, 257, 257, 257, 7: 257, 473: 4833, 480: 257, 1069: 4832}, + {2422, 2422, 2422, 2422, 7: 2422, 480: 2422}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3389, 651: 3391, 2675, 2676, 2674, 726: 3388, 856: 4834}, // 2375 {256, 256, 256, 256, 7: 256, 480: 256}, - {2422, 2422, 2422, 2422, 7: 2422, 480: 2422}, - {366: 4836}, - {496: 2648, 725: 2647, 734: 4837}, - {2426, 2426, 2426, 2426, 7: 2426, 201: 4838, 480: 2426, 1223: 4839}, - // 2380 - {254: 4840}, {2423, 2423, 2423, 2423, 7: 2423, 480: 2423}, - {464: 4842, 1286: 4841}, - {2425, 2425, 2425, 2425, 7: 4843, 480: 2425}, - {255, 255, 255, 255, 7: 255, 480: 255}, + {267: 4837}, + {497: 2649, 721: 2648, 731: 4838}, + {2427, 2427, 2427, 2427, 7: 2427, 176: 4839, 480: 2427, 1056: 4840}, + // 2380 + {254: 4841}, + {2424, 2424, 2424, 2424, 7: 2424, 480: 2424}, + {464: 4843, 1284: 4842}, + {2426, 2426, 2426, 2426, 7: 4844, 14: 2426, 16: 2426, 19: 2426, 468: 2426, 473: 2426, 480: 2426, 493: 2426, 495: 2426, 635: 2426}, + {255, 255, 255, 255, 7: 255, 14: 255, 16: 255, 19: 255, 468: 255, 473: 255, 480: 255, 493: 255, 495: 255, 635: 255}, // 2385 - {464: 4844}, - {254, 254, 254, 254, 7: 254, 480: 254}, + {464: 4845}, + {254, 254, 254, 254, 7: 254, 14: 254, 16: 254, 19: 254, 468: 254, 473: 254, 480: 254, 493: 254, 495: 254, 635: 254}, {6: 388, 27: 388}, - {382, 382, 382, 382, 382, 382, 382, 382, 13: 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 461: 382, 382, 382, 467: 382, 382, 382, 473: 382, 479: 382, 382, 491: 382, 382, 494: 382, 382, 559: 382, 636: 382, 382, 639: 382}, - {4: 4132, 4134, 389, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 468: 4131, 4168, 473: 2118, 492: 4845, 494: 2118, 637: 2118, 639: 4137, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4848}, + {382, 382, 382, 382, 382, 382, 382, 382, 13: 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 382, 461: 382, 382, 382, 467: 382, 382, 382, 473: 382, 479: 382, 382, 491: 382, 382, 495: 382, 382, 559: 382, 634: 382, 382, 637: 382}, + {4: 4133, 4135, 389, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 468: 4132, 4169, 473: 2118, 492: 4846, 495: 2118, 635: 2118, 637: 4138, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4849}, // 2390 - {381, 381, 381, 381, 381, 381, 381, 381, 13: 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 461: 381, 381, 381, 467: 381, 381, 381, 473: 381, 479: 381, 381, 491: 381, 381, 494: 381, 381, 559: 381, 636: 381, 381, 639: 381}, - {464: 4851, 468: 4850}, + {381, 381, 381, 381, 381, 381, 381, 381, 13: 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 381, 461: 381, 381, 381, 467: 381, 381, 381, 473: 381, 479: 381, 381, 491: 381, 381, 495: 381, 381, 559: 381, 634: 381, 381, 637: 381}, + {464: 4852, 468: 4851}, + {2434, 2434, 2434, 2434, 7: 2434, 480: 2434}, {2433, 2433, 2433, 2433, 7: 2433, 480: 2433}, - {2432, 2432, 2432, 2432, 7: 2432, 480: 2432}, - {464: 4854, 468: 4853}, + {464: 4855, 468: 4854}, // 2395 + {2436, 2436, 2436, 2436, 7: 2436, 480: 2436}, {2435, 2435, 2435, 2435, 7: 2435, 480: 2435}, - {2434, 2434, 2434, 2434, 7: 2434, 480: 2434}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 468: 1999, 487: 4172, 501: 4857, 726: 4856}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 4859, 468: 4861, 653: 4862, 2674, 2675, 2673, 872: 4860}, - {468: 4858}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 468: 1999, 487: 4173, 493: 4858, 724: 4857}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 4860, 468: 4862, 651: 4863, 2675, 2676, 2674, 870: 4861}, + {468: 4859}, // 2400 - {2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 13: 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 2436, 50: 2436, 461: 2436, 2436, 2436, 467: 2436, 2436, 2436, 473: 2436, 479: 2436, 2436, 491: 2436, 2436, 494: 2436, 2436, 559: 2436, 636: 2436, 2436, 639: 2436}, - {2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 13: 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 50: 2439, 461: 2439, 2439, 2439, 467: 2439, 2439, 2439, 473: 2439, 479: 2439, 2439, 491: 2439, 2439, 494: 2439, 2439, 559: 2439, 636: 2439, 2439, 639: 2439}, - {2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 13: 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 50: 2438, 461: 2438, 2438, 2438, 467: 2438, 2438, 2438, 473: 2438, 479: 2438, 2438, 491: 2438, 2438, 494: 2438, 2438, 559: 2438, 636: 2438, 2438, 639: 2438}, - {2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 13: 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 50: 2437, 461: 2437, 2437, 2437, 467: 2437, 2437, 2437, 473: 2437, 479: 2437, 2437, 491: 2437, 2437, 494: 2437, 2437, 559: 2437, 636: 2437, 2437, 639: 2437}, - {2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 13: 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 50: 2133, 86: 2133, 88: 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 2133, 461: 2133, 2133, 2133, 467: 2133, 2133, 2133, 473: 2133, 479: 2133, 2133, 491: 2133, 2133, 494: 2133, 2133, 559: 2133, 636: 2133, 2133, 639: 2133}, + {2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 13: 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 2437, 50: 2437, 461: 2437, 2437, 2437, 467: 2437, 2437, 2437, 473: 2437, 479: 2437, 2437, 491: 2437, 2437, 2437, 495: 2437, 2437, 559: 2437, 634: 2437, 2437, 637: 2437}, + {2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 13: 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 50: 2440, 461: 2440, 2440, 2440, 467: 2440, 2440, 2440, 473: 2440, 479: 2440, 2440, 491: 2440, 2440, 2440, 495: 2440, 2440, 559: 2440, 634: 2440, 2440, 637: 2440}, + {2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 13: 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 2439, 50: 2439, 461: 2439, 2439, 2439, 467: 2439, 2439, 2439, 473: 2439, 479: 2439, 2439, 491: 2439, 2439, 2439, 495: 2439, 2439, 559: 2439, 634: 2439, 2439, 637: 2439}, + {2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 13: 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 2438, 50: 2438, 461: 2438, 2438, 2438, 467: 2438, 2438, 2438, 473: 2438, 479: 2438, 2438, 491: 2438, 2438, 2438, 495: 2438, 2438, 559: 2438, 634: 2438, 2438, 637: 2438}, + {2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 13: 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 50: 2134, 86: 2134, 88: 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 2134, 461: 2134, 2134, 2134, 467: 2134, 2134, 2134, 473: 2134, 479: 2134, 2134, 491: 2134, 2134, 2134, 495: 2134, 2134, 559: 2134, 634: 2134, 2134, 637: 2134}, // 2405 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 4864}, - {2296, 2296, 7: 4627, 463: 4867, 639: 4866, 792: 4865}, - {2455, 2455}, - {880, 880, 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 880, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 4882}, - {496: 4872, 562: 3324, 3323, 725: 4870, 807: 4871, 975: 4869, 1151: 4868}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 4865}, + {2297, 2297, 7: 4628, 463: 4868, 637: 4867, 790: 4866}, + {2456, 2456}, + {880, 880, 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 880, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 4883}, + {497: 4873, 562: 3325, 3324, 721: 4871, 805: 4872, 973: 4870, 1150: 4869}, // 2410 - {2295, 2295, 7: 4880}, - {2294, 2294, 7: 2294}, - {221: 4874, 225: 4876, 270: 4877, 289: 4875}, - {185: 4873}, - {185: 2161, 221: 1930, 225: 1930, 270: 1930, 289: 1930}, + {2296, 2296, 7: 4881}, + {2295, 2295, 7: 2295}, + {221: 4875, 225: 4877, 271: 4878, 291: 4876}, + {186: 4874}, + {186: 2162, 221: 1930, 225: 1930, 271: 1930, 291: 1930}, // 2415 - {2287, 2287, 7: 2287}, - {2292, 2292, 7: 2292}, - {2291, 2291, 7: 2291}, - {315: 4878, 395: 4879}, {2288, 2288, 7: 2288}, + {2293, 2293, 7: 2293}, + {2292, 2292, 7: 2292}, + {317: 4879, 395: 4880}, + {2289, 2289, 7: 2289}, // 2420 + {2291, 2291, 7: 2291}, {2290, 2290, 7: 2290}, - {2289, 2289, 7: 2289}, - {496: 4872, 562: 3324, 3323, 725: 4870, 807: 4871, 975: 4881}, - {2293, 2293, 7: 2293}, - {2296, 2296, 7: 4886, 463: 4867, 792: 4885}, + {497: 4873, 562: 3325, 3324, 721: 4871, 805: 4872, 973: 4882}, + {2294, 2294, 7: 2294}, + {2297, 2297, 7: 4887, 463: 4868, 790: 4886}, // 2425 {879, 879, 7: 879, 50: 879, 463: 879}, {877, 877, 7: 877, 50: 877, 463: 877}, - {2454, 2454}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 576: 4888, 653: 4887, 2674, 2675, 2673}, + {2455, 2455}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 576: 4889, 651: 4888, 2675, 2676, 2674}, {878, 878, 7: 878, 50: 878, 463: 878}, // 2430 {876, 876, 7: 876, 50: 876, 463: 876}, - {2456, 2456}, - {2431, 2431}, - {355: 4956}, - {480: 4948}, + {2457, 2457}, + {2432, 2432}, + {357: 4957}, + {480: 4949}, // 2435 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 649: 4895, 653: 4894, 2674, 2675, 2673}, - {2084, 2084, 4: 2084, 2084, 16: 2084, 18: 2084, 2084, 2084, 22: 2084, 2084, 2084, 2084, 192: 4091, 639: 2084, 913: 4946, 935: 4947}, - {140: 2102, 341: 4900, 379: 4901, 516: 4899, 566: 2102, 1053: 4902, 4897, 1122: 4898, 1250: 4896}, - {2096, 2096, 100: 2096, 103: 4936, 461: 2096, 2096, 2096, 467: 2096, 479: 2096, 491: 2096, 495: 2096, 559: 2096, 636: 2096, 1251: 4935}, - {140: 4923, 566: 4922}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 647: 4896, 651: 4895, 2675, 2676, 2674}, + {2084, 2084, 4: 2084, 2084, 16: 2084, 18: 2084, 20: 2084, 2084, 2084, 2084, 2084, 2084, 193: 4092, 637: 2084, 911: 4947, 933: 4948}, + {140: 2102, 343: 4901, 380: 4902, 516: 4900, 566: 2102, 1051: 4903, 4898, 1121: 4899, 1248: 4897}, + {2096, 2096, 100: 2096, 103: 4937, 461: 2096, 2096, 2096, 467: 2096, 479: 2096, 491: 2096, 496: 2096, 559: 2096, 634: 2096, 1249: 4936}, + {140: 4924, 566: 4923}, // 2440 - {2110, 2110, 100: 2110, 103: 2110, 461: 2110, 2110, 2110, 467: 2110, 479: 2110, 491: 2110, 495: 2110, 559: 2110, 636: 2110}, - {98: 3841, 107: 3840, 462: 4915, 821: 4916}, - {98: 3841, 107: 3840, 462: 4908, 821: 4909}, - {2103, 2103, 100: 2103, 103: 2103, 461: 2103, 2103, 2103, 467: 2103, 479: 2103, 483: 4904, 491: 2103, 495: 2103, 559: 2103, 570: 4903, 636: 2103}, + {2110, 2110, 100: 2110, 103: 2110, 461: 2110, 2110, 2110, 467: 2110, 479: 2110, 491: 2110, 496: 2110, 559: 2110, 634: 2110}, + {98: 3842, 107: 3841, 462: 4916, 819: 4917}, + {98: 3842, 107: 3841, 462: 4909, 819: 4910}, + {2103, 2103, 100: 2103, 103: 2103, 461: 2103, 2103, 2103, 467: 2103, 479: 2103, 483: 4905, 491: 2103, 496: 2103, 559: 2103, 570: 4904, 634: 2103}, {140: 2101, 566: 2101}, // 2445 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 4906}, - {496: 2648, 725: 2647, 734: 4905}, - {2104, 2104, 100: 2104, 103: 2104, 461: 2104, 2104, 2104, 467: 2104, 479: 2104, 491: 2104, 495: 2104, 559: 2104, 636: 2104}, - {105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 493: 3247, 497: 3245, 3246, 3244, 3242, 520: 3353, 3350, 3352, 3351, 3347, 3349, 3348, 3345, 3346, 3344, 3354, 723: 3243, 3241, 794: 3343, 817: 4907}, - {2105, 2105, 100: 2105, 103: 2105, 461: 2105, 2105, 2105, 467: 2105, 479: 2105, 491: 2105, 495: 2105, 559: 2105, 636: 2105}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 4907}, + {497: 2649, 721: 2648, 731: 4906}, + {2104, 2104, 100: 2104, 103: 2104, 461: 2104, 2104, 2104, 467: 2104, 479: 2104, 491: 2104, 496: 2104, 559: 2104, 634: 2104}, + {105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 494: 3246, 498: 3244, 3245, 3243, 3241, 520: 3354, 3351, 3353, 3352, 3348, 3350, 3349, 3346, 3347, 3345, 3355, 722: 3242, 3240, 792: 3344, 815: 4908}, + {2105, 2105, 100: 2105, 103: 2105, 461: 2105, 2105, 2105, 467: 2105, 479: 2105, 491: 2105, 496: 2105, 559: 2105, 634: 2105}, // 2450 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4913}, - {462: 4910}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 4911}, - {7: 4041, 50: 4912}, - {2106, 2106, 100: 2106, 103: 2106, 461: 2106, 2106, 2106, 467: 2106, 479: 2106, 491: 2106, 495: 2106, 559: 2106, 636: 2106}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4914}, + {462: 4911}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 4912}, + {7: 4042, 50: 4913}, + {2106, 2106, 100: 2106, 103: 2106, 461: 2106, 2106, 2106, 467: 2106, 479: 2106, 491: 2106, 496: 2106, 559: 2106, 634: 2106}, // 2455 - {50: 4914, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2107, 2107, 100: 2107, 103: 2107, 461: 2107, 2107, 2107, 467: 2107, 479: 2107, 491: 2107, 495: 2107, 559: 2107, 636: 2107}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4920}, - {462: 4917}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 4918}, + {50: 4915, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2107, 2107, 100: 2107, 103: 2107, 461: 2107, 2107, 2107, 467: 2107, 479: 2107, 491: 2107, 496: 2107, 559: 2107, 634: 2107}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4921}, + {462: 4918}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 4919}, // 2460 - {7: 4041, 50: 4919}, - {2108, 2108, 100: 2108, 103: 2108, 461: 2108, 2108, 2108, 467: 2108, 479: 2108, 491: 2108, 495: 2108, 559: 2108, 636: 2108}, - {50: 4921, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2109, 2109, 100: 2109, 103: 2109, 461: 2109, 2109, 2109, 467: 2109, 479: 2109, 491: 2109, 495: 2109, 559: 2109, 636: 2109}, - {81: 4928, 462: 2112, 1249: 4927}, + {7: 4042, 50: 4920}, + {2108, 2108, 100: 2108, 103: 2108, 461: 2108, 2108, 2108, 467: 2108, 479: 2108, 491: 2108, 496: 2108, 559: 2108, 634: 2108}, + {50: 4922, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2109, 2109, 100: 2109, 103: 2109, 461: 2109, 2109, 2109, 467: 2109, 479: 2109, 491: 2109, 496: 2109, 559: 2109, 634: 2109}, + {81: 4929, 462: 2112, 1247: 4928}, // 2465 - {462: 4924}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 4925}, - {50: 4926, 474: 3586, 3585, 3591, 512: 3587, 545: 3588, 3589, 3582, 3592, 3581, 3590, 3583, 3584}, - {2113, 2113, 100: 2113, 103: 2113, 212: 2113, 461: 2113, 2113, 2113, 467: 2113, 479: 2113, 491: 2113, 495: 2113, 559: 2113, 636: 2113}, - {462: 4931}, + {462: 4925}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 4926}, + {50: 4927, 474: 3587, 3586, 3592, 512: 3588, 545: 3589, 3590, 3583, 3593, 3582, 3591, 3584, 3585}, + {2113, 2113, 100: 2113, 103: 2113, 212: 2113, 461: 2113, 2113, 2113, 467: 2113, 479: 2113, 491: 2113, 496: 2113, 559: 2113, 634: 2113}, + {462: 4932}, // 2470 - {487: 4929}, - {496: 2648, 725: 4930}, + {487: 4930}, + {497: 2649, 721: 4931}, {462: 2111}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 2262, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 4932, 985: 4933}, - {7: 4041, 50: 2261}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 2263, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 4933, 983: 4934}, + {7: 4042, 50: 2262}, // 2475 - {50: 4934}, - {2114, 2114, 100: 2114, 103: 2114, 212: 2114, 461: 2114, 2114, 2114, 467: 2114, 479: 2114, 491: 2114, 495: 2114, 559: 2114, 636: 2114}, - {2100, 2100, 100: 4939, 461: 2100, 2100, 2100, 467: 2100, 479: 2100, 491: 2100, 495: 2100, 559: 2100, 636: 2100, 1292: 4938}, - {496: 2648, 725: 2647, 734: 4937}, - {2095, 2095, 100: 2095, 461: 2095, 2095, 2095, 467: 2095, 479: 2095, 491: 2095, 495: 2095, 559: 2095, 636: 2095}, + {50: 4935}, + {2114, 2114, 100: 2114, 103: 2114, 212: 2114, 461: 2114, 2114, 2114, 467: 2114, 479: 2114, 491: 2114, 496: 2114, 559: 2114, 634: 2114}, + {2100, 2100, 100: 4940, 461: 2100, 2100, 2100, 467: 2100, 479: 2100, 491: 2100, 496: 2100, 559: 2100, 634: 2100, 1290: 4939}, + {497: 2649, 721: 2648, 731: 4938}, + {2095, 2095, 100: 2095, 461: 2095, 2095, 2095, 467: 2095, 479: 2095, 491: 2095, 496: 2095, 559: 2095, 634: 2095}, // 2480 - {2094, 2094, 461: 2094, 4774, 2094, 467: 2094, 479: 2094, 491: 2094, 495: 2094, 559: 2094, 636: 2094, 1079: 4945}, - {649: 4940}, - {140: 2102, 566: 2102, 1053: 4902, 4897, 1122: 4941}, - {2098, 2098, 212: 4943, 461: 2098, 2098, 2098, 467: 2098, 479: 2098, 491: 2098, 495: 2098, 559: 2098, 636: 2098, 1291: 4942}, - {2099, 2099, 461: 2099, 2099, 2099, 467: 2099, 479: 2099, 491: 2099, 495: 2099, 559: 2099, 636: 2099}, + {2094, 2094, 461: 2094, 4775, 2094, 467: 2094, 479: 2094, 491: 2094, 496: 2094, 559: 2094, 634: 2094, 1078: 4946}, + {647: 4941}, + {140: 2102, 566: 2102, 1051: 4903, 4898, 1121: 4942}, + {2098, 2098, 212: 4944, 461: 2098, 2098, 2098, 467: 2098, 479: 2098, 491: 2098, 496: 2098, 559: 2098, 634: 2098, 1289: 4943}, + {2099, 2099, 461: 2099, 2099, 2099, 467: 2099, 479: 2099, 491: 2099, 496: 2099, 559: 2099, 634: 2099}, // 2485 - {496: 2648, 725: 2647, 734: 4944}, - {2097, 2097, 461: 2097, 2097, 2097, 467: 2097, 479: 2097, 491: 2097, 495: 2097, 559: 2097, 636: 2097}, - {2115, 2115, 461: 2115, 2115, 2115, 467: 2115, 479: 2115, 491: 2115, 495: 2115, 559: 2115, 636: 2115}, - {2428, 2428}, - {2427, 2427, 4: 4132, 4803, 16: 4090, 18: 4140, 4133, 4136, 22: 4135, 4138, 4139, 4141, 639: 4137, 766: 4142, 800: 4802}, - // 2490 - {550, 550, 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4622, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 806: 4621, 824: 4949}, - {2369, 2369, 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 4951, 1259: 4950}, + {497: 2649, 721: 2648, 731: 4945}, + {2097, 2097, 461: 2097, 2097, 2097, 467: 2097, 479: 2097, 491: 2097, 496: 2097, 559: 2097, 634: 2097}, + {2115, 2115, 461: 2115, 2115, 2115, 467: 2115, 479: 2115, 491: 2115, 496: 2115, 559: 2115, 634: 2115}, {2429, 2429}, - {7: 4627, 484: 4952}, - {462: 4953}, - // 2495 - {480: 4777, 936: 4776, 1078: 4954}, - {7: 4813, 50: 4955}, - {2368, 2368}, + {2428, 2428, 4: 4133, 4804, 16: 4091, 18: 4141, 20: 4134, 4137, 4136, 4139, 4140, 4142, 637: 4138, 764: 4143, 798: 4803}, + // 2490 + {550, 550, 550, 550, 550, 550, 550, 8: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 51: 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 4623, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 550, 804: 4622, 822: 4950}, + {2370, 2370, 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 4952, 1257: 4951}, {2430, 2430}, - {136: 4958, 867: 96, 1057: 4959}, + {7: 4628, 484: 4953}, + {462: 4954}, + // 2495 + {480: 4778, 934: 4777, 1077: 4955}, + {7: 4814, 50: 4956}, + {2369, 2369}, + {2431, 2431}, + {136: 4959, 865: 96, 1055: 4960}, // 2500 - {867: 95}, - {867: 4960}, - {464: 4961}, - {17, 17, 176: 17, 343: 4963, 648: 17, 1229: 4962}, - {15, 15, 176: 4966, 648: 15, 1228: 4965}, + {865: 95}, + {865: 4961}, + {464: 4962}, + {17, 17, 177: 17, 345: 4964, 646: 17, 1227: 4963}, + {15, 15, 177: 4967, 646: 15, 1226: 4966}, // 2505 - {496: 2648, 725: 4964}, - {16, 16, 176: 16, 648: 16}, - {81, 81, 648: 3862, 931: 4973}, - {13, 13, 180: 13, 356: 4968, 648: 13, 1253: 4967}, - {11, 11, 180: 4971, 648: 11, 1252: 4970}, + {497: 2649, 721: 4965}, + {16, 16, 177: 16, 646: 16}, + {81, 81, 646: 3863, 929: 4974}, + {13, 13, 181: 13, 358: 4969, 646: 13, 1251: 4968}, + {11, 11, 181: 4972, 646: 11, 1250: 4971}, // 2510 - {496: 2648, 725: 4969}, - {12, 12, 180: 12, 648: 12}, - {14, 14, 648: 14}, - {496: 2648, 725: 4972}, - {10, 10, 648: 10}, + {497: 2649, 721: 4970}, + {12, 12, 181: 12, 646: 12}, + {14, 14, 646: 14}, + {497: 2649, 721: 4973}, + {10, 10, 646: 10}, // 2515 {18, 18}, - {26: 55, 143: 55, 496: 55}, + {26: 55, 143: 55, 497: 55}, {59, 59}, - {496: 2648, 725: 4979}, - {496: 2648, 725: 4978}, + {497: 2649, 721: 4980}, + {497: 2649, 721: 4979}, // 2520 {57, 57}, {58, 58}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4984, 1125: 4985, 1294: 4983}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4985, 1124: 4986, 1292: 4984}, {68, 68, 68, 68, 68, 68, 68, 8: 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 51: 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 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, 8: 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 51: 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, // 2525 - {69, 69, 7: 4991}, - {661: 4987, 677: 4988, 1224: 4986}, + {69, 69, 7: 4992}, + {659: 4988, 675: 4989, 1222: 4987}, {61, 61, 7: 61}, {66, 66, 7: 66}, - {65, 65, 7: 65, 136: 4990}, + {65, 65, 7: 65, 136: 4991}, // 2530 - {63, 63, 7: 63, 136: 4989}, + {63, 63, 7: 63, 136: 4990}, {62, 62, 7: 62}, {64, 64, 7: 64}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 4984, 1125: 4992}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 4985, 1124: 4993}, {60, 60, 7: 60}, // 2535 {70, 70}, - {136: 4958, 867: 96, 1057: 4997}, - {464: 4996}, + {136: 4959, 865: 96, 1055: 4998}, + {464: 4997}, {54, 54}, - {867: 4998}, + {865: 4999}, // 2540 - {464: 4999}, - {479: 5000, 484: 2064, 495: 5001, 1021: 5002}, - {2063, 2063, 461: 2063, 2063, 2063, 467: 2063, 484: 2063, 491: 2063, 559: 2063, 636: 2063}, - {2062, 2062, 461: 2062, 2062, 2062, 467: 2062, 484: 2062, 491: 2062, 559: 2062, 636: 2062}, - {484: 5003}, + {464: 5000}, + {479: 5001, 484: 2064, 496: 5002, 1019: 5003}, + {2063, 2063, 461: 2063, 2063, 2063, 467: 2063, 484: 2063, 491: 2063, 559: 2063, 634: 2063}, + {2062, 2062, 461: 2062, 2062, 2062, 467: 2062, 484: 2062, 491: 2062, 559: 2062, 634: 2062}, + {484: 5004}, // 2545 - {559: 5004}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5005}, - {98, 98, 98: 98, 107: 98, 462: 98, 479: 98, 501: 98, 637: 5007, 648: 98, 1163: 5006}, - {94, 94, 98: 3841, 107: 3840, 462: 94, 479: 94, 501: 94, 648: 94, 821: 3839, 1031: 5010}, - {501: 5008}, + {559: 5005}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5006}, + {98, 98, 98: 98, 107: 98, 462: 98, 479: 98, 493: 98, 635: 5008, 646: 98, 1162: 5007}, + {94, 94, 98: 3842, 107: 3841, 462: 94, 479: 94, 493: 94, 646: 94, 819: 3840, 1029: 5011}, + {493: 5009}, // 2550 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 5009}, - {97, 97, 98: 97, 107: 97, 462: 97, 479: 97, 501: 97, 648: 97}, - {81, 81, 462: 81, 479: 81, 501: 81, 648: 3862, 931: 5011}, - {100, 100, 462: 100, 479: 5013, 501: 100, 1205: 5012}, - {2250, 2250, 462: 5016, 501: 2250, 1169: 5017}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 5010}, + {97, 97, 98: 97, 107: 97, 462: 97, 479: 97, 493: 97, 646: 97}, + {81, 81, 462: 81, 479: 81, 493: 81, 646: 3863, 929: 5012}, + {100, 100, 462: 100, 479: 5014, 493: 100, 1204: 5013}, + {2251, 2251, 462: 5017, 493: 2251, 1168: 5018}, // 2555 - {496: 2648, 725: 5014}, - {648: 5015}, - {99, 99, 462: 99, 501: 99}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 2256, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 557: 3250, 653: 4038, 2674, 2675, 2673, 700: 5030, 733: 5029, 986: 5028, 1167: 5027, 5031}, - {75, 75, 501: 5019, 1222: 5018}, + {497: 2649, 721: 5015}, + {646: 5016}, + {99, 99, 462: 99, 493: 99}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 2257, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 557: 3249, 651: 4039, 2675, 2676, 2674, 698: 5031, 732: 5030, 984: 5029, 1166: 5028, 5032}, + {75, 75, 493: 5020, 1221: 5019}, // 2560 {101, 101}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3677, 2674, 2675, 2673, 701: 5022, 1055: 5021, 1221: 5020}, - {74, 74, 7: 5025}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3678, 2675, 2676, 2674, 699: 5023, 1053: 5022, 1220: 5021}, + {74, 74, 7: 5026}, {72, 72, 7: 72}, - {487: 5023}, + {487: 5024}, // 2565 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5024}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5025}, {71, 71, 7: 71}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3677, 2674, 2675, 2673, 701: 5022, 1055: 5026}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3678, 2675, 2676, 2674, 699: 5023, 1053: 5027}, {73, 73, 7: 73}, - {7: 5033, 50: 2255}, + {7: 5034, 50: 2256}, // 2570 - {7: 2254, 50: 2254}, + {7: 2255, 50: 2255}, + {7: 2253, 50: 2253}, {7: 2252, 50: 2252}, - {7: 2251, 50: 2251}, - {50: 5032}, - {2249, 2249, 501: 2249}, + {50: 5033}, + {2250, 2250, 493: 2250}, // 2575 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 557: 3250, 653: 4038, 2674, 2675, 2673, 700: 5030, 733: 5029, 986: 5034}, - {7: 2253, 50: 2253}, - {7: 158, 163: 158, 461: 158, 486: 158, 557: 1789, 640: 158, 657: 1789}, - {7: 123, 461: 123, 123, 486: 123, 557: 1760, 640: 123, 657: 1760}, - {7: 137, 461: 137, 137, 486: 137, 557: 1734, 640: 137, 657: 1734}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 557: 3249, 651: 4039, 2675, 2676, 2674, 698: 5031, 732: 5030, 984: 5035}, + {7: 2254, 50: 2254}, + {7: 158, 163: 158, 461: 158, 486: 158, 557: 1789, 638: 158, 655: 1789}, + {7: 123, 461: 123, 123, 486: 123, 557: 1760, 638: 123, 655: 1760}, + {7: 137, 461: 137, 137, 486: 137, 557: 1734, 638: 137, 655: 1734}, // 2580 - {7: 124, 461: 124, 124, 486: 124, 557: 1731, 640: 124, 657: 1731}, - {7: 113, 461: 113, 113, 486: 113, 557: 1696, 640: 113, 657: 1696}, - {7: 133, 461: 133, 133, 486: 133, 557: 1621, 640: 133, 657: 1621}, - {7: 138, 461: 138, 138, 486: 138, 557: 1614, 640: 138, 657: 1614}, - {307: 5144, 372: 5143, 557: 1596, 657: 1596}, + {7: 124, 461: 124, 124, 486: 124, 557: 1731, 638: 124, 655: 1731}, + {7: 113, 461: 113, 113, 486: 113, 557: 1696, 638: 113, 655: 1696}, + {7: 133, 461: 133, 133, 486: 133, 557: 1621, 638: 133, 655: 1621}, + {7: 138, 461: 138, 138, 486: 138, 557: 1614, 638: 138, 655: 1614}, + {309: 5145, 373: 5144, 557: 1596, 655: 1596}, // 2585 - {7: 125, 461: 125, 125, 486: 125, 557: 1593, 640: 125, 657: 1593}, - {7: 114, 461: 114, 114, 486: 114, 557: 1590, 640: 114, 657: 1590}, - {557: 5141, 657: 5140}, - {7: 707, 461: 707, 486: 707, 557: 248, 640: 707, 657: 248}, - {7: 706, 461: 706, 486: 706, 640: 706}, + {7: 125, 461: 125, 125, 486: 125, 557: 1593, 638: 125, 655: 1593}, + {7: 114, 461: 114, 114, 486: 114, 557: 1590, 638: 114, 655: 1590}, + {557: 5142, 655: 5141}, + {7: 707, 461: 707, 486: 707, 557: 248, 638: 707, 655: 248}, + {7: 706, 461: 706, 486: 706, 638: 706}, // 2590 - {7: 154, 163: 5139, 461: 154, 486: 154, 640: 154}, - {7: 156, 461: 156, 486: 156, 640: 156}, - {7: 155, 461: 155, 486: 155, 640: 155}, - {486: 5137}, - {7: 134, 461: 134, 134, 484: 5135, 486: 134, 640: 134}, + {7: 154, 163: 5140, 461: 154, 486: 154, 638: 154}, + {7: 156, 461: 156, 486: 156, 638: 156}, + {7: 155, 461: 155, 486: 155, 638: 155}, + {486: 5138}, + {7: 134, 461: 134, 134, 484: 5136, 486: 134, 638: 134}, // 2595 - {7: 151, 461: 151, 486: 151, 640: 151}, - {7: 5087, 461: 5088, 486: 5089}, - {7: 149, 461: 149, 5084, 486: 149, 640: 149}, - {7: 147, 181: 5083, 461: 147, 147, 486: 147, 640: 147}, - {7: 145, 268: 5082, 461: 145, 145, 486: 145, 640: 145}, + {7: 151, 461: 151, 486: 151, 638: 151}, + {7: 5088, 461: 5089, 486: 5090}, + {7: 149, 461: 149, 5085, 486: 149, 638: 149}, + {7: 147, 182: 5084, 461: 147, 147, 486: 147, 638: 147}, + {7: 145, 269: 5083, 461: 145, 145, 486: 145, 638: 145}, // 2600 - {7: 144, 18: 5076, 99: 5078, 160: 5077, 162: 5075, 167: 5079, 268: 5080, 461: 144, 144, 486: 144, 640: 144}, - {7: 141, 461: 141, 141, 486: 141, 640: 141}, - {7: 140, 461: 140, 140, 486: 140, 640: 140}, - {7: 139, 167: 5074, 461: 139, 139, 486: 139, 640: 139}, - {7: 136, 461: 136, 136, 486: 136, 640: 136}, + {7: 144, 18: 5077, 99: 5079, 160: 5078, 162: 5076, 167: 5080, 269: 5081, 461: 144, 144, 486: 144, 638: 144}, + {7: 141, 461: 141, 141, 486: 141, 638: 141}, + {7: 140, 461: 140, 140, 486: 140, 638: 140}, + {7: 139, 167: 5075, 461: 139, 139, 486: 139, 638: 139}, + {7: 136, 461: 136, 136, 486: 136, 638: 136}, // 2605 - {7: 135, 461: 135, 135, 486: 135, 640: 135}, - {99: 5073, 1004: 5072}, - {7: 131, 461: 131, 131, 486: 131, 640: 131}, - {893: 5071}, - {7: 129, 461: 129, 129, 486: 129, 640: 129}, + {7: 135, 461: 135, 135, 486: 135, 638: 135}, + {99: 5074, 1002: 5073}, + {7: 131, 461: 131, 131, 486: 131, 638: 131}, + {891: 5072}, + {7: 129, 461: 129, 129, 486: 129, 638: 129}, // 2610 - {7: 126, 461: 126, 126, 486: 126, 640: 126}, - {110: 5070}, - {7: 121, 461: 121, 121, 486: 121, 640: 121}, - {7: 130, 461: 130, 130, 486: 130, 640: 130}, - {7: 132, 461: 132, 132, 486: 132, 640: 132}, + {7: 126, 461: 126, 126, 486: 126, 638: 126}, + {110: 5071}, + {7: 121, 461: 121, 121, 486: 121, 638: 121}, + {7: 130, 461: 130, 130, 486: 130, 638: 130}, + {7: 132, 461: 132, 132, 486: 132, 638: 132}, // 2615 - {7: 119, 461: 119, 119, 486: 119, 640: 119}, - {7: 117, 461: 117, 117, 486: 117, 640: 117}, - {7: 143, 461: 143, 143, 486: 143, 640: 143}, - {7: 142, 461: 142, 142, 486: 142, 640: 142}, - {110: 5081}, + {7: 119, 461: 119, 119, 486: 119, 638: 119}, + {7: 117, 461: 117, 117, 486: 117, 638: 117}, + {7: 143, 461: 143, 143, 486: 143, 638: 143}, + {7: 142, 461: 142, 142, 486: 142, 638: 142}, + {110: 5082}, // 2620 - {7: 120, 461: 120, 120, 486: 120, 640: 120}, - {7: 118, 461: 118, 118, 486: 118, 640: 118}, - {7: 116, 461: 116, 116, 486: 116, 640: 116}, - {7: 122, 461: 122, 122, 486: 122, 640: 122}, - {7: 115, 461: 115, 115, 486: 115, 640: 115}, + {7: 120, 461: 120, 120, 486: 120, 638: 120}, + {7: 118, 461: 118, 118, 486: 118, 638: 118}, + {7: 116, 461: 116, 116, 486: 116, 638: 116}, + {7: 122, 461: 122, 122, 486: 122, 638: 122}, + {7: 115, 461: 115, 115, 486: 115, 638: 115}, // 2625 - {7: 146, 461: 146, 146, 486: 146, 640: 146}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 5085}, - {7: 4041, 50: 5086}, - {7: 148, 461: 148, 486: 148, 640: 148}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5035, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 5037, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 5043, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 5039, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 5036, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 5044, 3108, 2841, 3061, 5038, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 5041, 2754, 2755, 2991, 5042, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 5040, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5046, 485: 5069, 556: 5063, 634: 5067, 636: 5052, 639: 5062, 641: 5056, 644: 5065, 652: 5057, 3390, 2674, 2675, 2673, 659: 5061, 664: 5058, 728: 5045, 732: 5060, 789: 5047, 797: 5051, 841: 5066, 852: 5064, 923: 5048, 941: 5049, 5055, 947: 5050, 5134, 956: 5059, 958: 5068}, + {7: 146, 461: 146, 146, 486: 146, 638: 146}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 5086}, + {7: 4042, 50: 5087}, + {7: 148, 461: 148, 486: 148, 638: 148}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5036, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 5038, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 5044, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 5040, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 5037, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 5045, 3109, 2842, 3062, 5039, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 5042, 2755, 2756, 2992, 5043, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 5041, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5047, 485: 5070, 556: 5064, 632: 5068, 634: 5053, 637: 5063, 639: 5057, 642: 5066, 650: 5058, 3391, 2675, 2676, 2674, 657: 5062, 662: 5059, 726: 5046, 730: 5061, 787: 5048, 795: 5052, 839: 5067, 850: 5065, 921: 5049, 939: 5050, 5056, 945: 5051, 5135, 954: 5060, 956: 5069}, // 2630 - {2: 112, 112, 112, 112, 112, 8: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 51: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5101, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 512: 112, 559: 5100, 943: 5102, 1064: 5103}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 5093}, - {720, 720, 7: 720, 13: 720, 46: 720, 99: 720, 141: 720, 463: 720, 470: 720, 487: 720, 557: 5098, 640: 720, 651: 720, 657: 5097, 720}, - {1175, 1175, 7: 1175, 13: 1175, 46: 1175, 99: 1175, 141: 1175, 462: 3667, 1175, 470: 1175, 487: 1175, 640: 1175, 651: 1175, 658: 1175, 1073: 5096}, + {2: 112, 112, 112, 112, 112, 8: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 51: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5102, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 512: 112, 559: 5101, 941: 5103, 1063: 5104}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 5094}, + {720, 720, 7: 720, 13: 720, 46: 720, 99: 720, 141: 720, 463: 720, 470: 720, 487: 720, 557: 5099, 638: 720, 649: 720, 655: 5098, 720}, + {1175, 1175, 7: 1175, 13: 1175, 46: 1175, 99: 1175, 141: 1175, 462: 3668, 1175, 470: 1175, 487: 1175, 638: 1175, 649: 1175, 656: 1175, 1072: 5097}, {716, 716, 7: 716, 463: 716}, // 2635 - {102, 102, 7: 5094}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5095}, + {102, 102, 7: 5095}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5096}, {715, 715, 7: 715, 463: 715}, - {717, 717, 7: 717, 13: 717, 46: 717, 99: 717, 141: 717, 463: 717, 470: 717, 487: 717, 640: 717, 651: 717, 658: 717}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 5099}, + {717, 717, 7: 717, 13: 717, 46: 717, 99: 717, 141: 717, 463: 717, 470: 717, 487: 717, 638: 717, 649: 717, 656: 717}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 5100}, // 2640 - {718, 718, 7: 718, 13: 718, 46: 718, 99: 718, 141: 718, 463: 718, 470: 718, 487: 718, 640: 718, 651: 718, 658: 718}, - {719, 719, 7: 719, 13: 719, 46: 719, 99: 719, 141: 719, 463: 719, 470: 719, 487: 719, 640: 719, 651: 719, 658: 719}, + {718, 718, 7: 718, 13: 718, 46: 718, 99: 718, 141: 718, 463: 718, 470: 718, 487: 718, 638: 718, 649: 718, 656: 718}, + {719, 719, 7: 719, 13: 719, 46: 719, 99: 719, 141: 719, 463: 719, 470: 719, 487: 719, 638: 719, 649: 719, 656: 719}, {2: 111, 111, 111, 111, 111, 8: 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 51: 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 111, 512: 111}, {2: 110, 110, 110, 110, 110, 8: 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 51: 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 110, 512: 110}, {2: 109, 109, 109, 109, 109, 8: 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 51: 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 109, 512: 109}, // 2645 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5104, 653: 5105, 2674, 2675, 2673, 1087: 5106}, - {486: 108, 640: 108, 642: 5132}, - {486: 104, 640: 104, 642: 5129}, - {486: 5107}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5108, 877: 5109, 909: 5110}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5105, 651: 5106, 2675, 2676, 2674, 1086: 5107}, + {486: 108, 638: 108, 640: 5133}, + {486: 104, 638: 104, 640: 5130}, + {486: 5108}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5109, 875: 5110, 907: 5111}, // 2650 - {190, 190, 7: 190, 13: 190, 46: 190, 141: 5114, 463: 190, 651: 190, 1155: 5113}, - {225, 225, 7: 225, 13: 225, 46: 225, 463: 225, 651: 225}, - {103, 103, 7: 5111}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5108, 877: 5112}, - {224, 224, 7: 224, 13: 224, 46: 224, 463: 224, 651: 224}, + {190, 190, 7: 190, 13: 190, 46: 190, 141: 5115, 463: 190, 649: 190, 1154: 5114}, + {225, 225, 7: 225, 13: 225, 46: 225, 463: 225, 649: 225}, + {103, 103, 7: 5112}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5109, 875: 5113}, + {224, 224, 7: 224, 13: 224, 46: 224, 463: 224, 649: 224}, // 2655 - {226, 226, 7: 226, 13: 226, 46: 226, 463: 226, 651: 226}, - {463: 5116, 649: 5115}, - {13: 5127, 464: 5124, 879: 5126}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 5118, 1156: 5117}, - {188, 188, 7: 188, 13: 188, 46: 188, 463: 188, 467: 5120, 649: 5119, 651: 188}, + {226, 226, 7: 226, 13: 226, 46: 226, 463: 226, 649: 226}, + {463: 5117, 647: 5116}, + {13: 5128, 464: 5125, 877: 5127}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 5119, 1155: 5118}, + {188, 188, 7: 188, 13: 188, 46: 188, 463: 188, 467: 5121, 647: 5120, 649: 188}, // 2660 - {184, 184, 7: 184, 13: 184, 46: 184, 463: 184, 467: 184, 649: 184, 651: 184}, - {464: 5124, 879: 5125}, - {464: 5122, 565: 5123, 1040: 5121}, - {186, 186, 7: 186, 13: 186, 46: 186, 463: 186, 651: 186}, - {183, 183, 7: 183, 13: 183, 46: 183, 463: 183, 651: 183}, + {184, 184, 7: 184, 13: 184, 46: 184, 463: 184, 467: 184, 647: 184, 649: 184}, + {464: 5125, 877: 5126}, + {464: 5123, 565: 5124, 1038: 5122}, + {186, 186, 7: 186, 13: 186, 46: 186, 463: 186, 649: 186}, + {183, 183, 7: 183, 13: 183, 46: 183, 463: 183, 649: 183}, // 2665 - {182, 182, 7: 182, 13: 182, 46: 182, 463: 182, 651: 182}, - {712, 712, 7: 712, 13: 712, 46: 712, 50: 712, 463: 712, 651: 712}, - {187, 187, 7: 187, 13: 187, 46: 187, 463: 187, 651: 187}, - {189, 189, 7: 189, 13: 189, 46: 189, 463: 189, 651: 189}, - {464: 5122, 565: 5123, 1040: 5128}, + {182, 182, 7: 182, 13: 182, 46: 182, 463: 182, 649: 182}, + {712, 712, 7: 712, 13: 712, 46: 712, 50: 712, 463: 712, 649: 712}, + {187, 187, 7: 187, 13: 187, 46: 187, 463: 187, 649: 187}, + {189, 189, 7: 189, 13: 189, 46: 189, 463: 189, 649: 189}, + {464: 5123, 565: 5124, 1038: 5129}, // 2670 - {185, 185, 7: 185, 13: 185, 46: 185, 463: 185, 651: 185}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5130, 653: 5131, 2674, 2675, 2673}, - {486: 106, 640: 106}, - {486: 105, 640: 105}, - {512: 5133}, + {185, 185, 7: 185, 13: 185, 46: 185, 463: 185, 649: 185}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5131, 651: 5132, 2675, 2676, 2674}, + {486: 106, 638: 106}, + {486: 105, 638: 105}, + {512: 5134}, // 2675 - {486: 107, 640: 107}, - {7: 150, 461: 150, 486: 150, 640: 150}, - {269: 5136}, - {7: 152, 461: 152, 486: 152, 640: 152}, - {269: 5138}, + {486: 107, 638: 107}, + {7: 150, 461: 150, 486: 150, 638: 150}, + {270: 5137}, + {7: 152, 461: 152, 486: 152, 638: 152}, + {270: 5139}, // 2680 - {7: 153, 461: 153, 486: 153, 640: 153}, - {7: 157, 163: 157, 461: 157, 486: 157, 640: 157}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 653: 3390, 2674, 2675, 2673, 728: 5142}, - {708, 708, 7: 708, 461: 708, 486: 708, 640: 708}, - {709, 709, 7: 709, 461: 709, 486: 709, 640: 709}, + {7: 153, 461: 153, 486: 153, 638: 153}, + {7: 157, 163: 157, 461: 157, 486: 157, 638: 157}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 651: 3391, 2675, 2676, 2674, 726: 5143}, + {708, 708, 7: 708, 461: 708, 486: 708, 638: 708}, + {709, 709, 7: 709, 461: 709, 486: 709, 638: 709}, // 2685 - {7: 128, 461: 128, 128, 486: 128, 640: 128}, - {7: 127, 461: 127, 127, 486: 127, 640: 127}, - {461: 5185, 557: 1707, 657: 1707}, - {7: 5087, 461: 5147, 640: 5148}, - {2: 112, 112, 112, 112, 112, 8: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 51: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5101, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 512: 112, 559: 5100, 943: 5102, 1064: 5150}, + {7: 128, 461: 128, 128, 486: 128, 638: 128}, + {7: 127, 461: 127, 127, 486: 127, 638: 127}, + {461: 5186, 557: 1707, 655: 1707}, + {7: 5088, 461: 5148, 638: 5149}, + {2: 112, 112, 112, 112, 112, 8: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 51: 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 5102, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 112, 512: 112, 559: 5101, 941: 5103, 1063: 5151}, // 2690 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 5149}, - {165, 165, 7: 5094}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5104, 653: 5105, 2674, 2675, 2673, 1087: 5151}, - {640: 5152}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5108, 877: 5109, 909: 5153}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 5150}, + {165, 165, 7: 5095}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5105, 651: 5106, 2675, 2676, 2674, 1086: 5152}, + {638: 5153}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5109, 875: 5110, 907: 5154}, // 2695 - {215, 215, 7: 5111, 463: 215, 651: 5155, 944: 5154, 5156}, + {215, 215, 7: 5112, 463: 215, 649: 5156, 942: 5155, 5157}, {214, 214, 13: 214, 46: 214, 463: 214}, - {131: 5176, 133: 5174, 5177, 5175, 348: 5169, 396: 5171, 946: 5173, 1260: 5172, 1278: 5170}, - {164, 164, 463: 5158, 1142: 5157}, + {131: 5177, 133: 5175, 5178, 5176, 350: 5170, 396: 5172, 944: 5174, 1258: 5173, 1276: 5171}, + {164, 164, 463: 5159, 1141: 5158}, {167, 167}, // 2700 - {126: 5162, 5160, 5161, 5163, 841: 5159}, - {893: 5168}, - {496: 2648, 725: 5167}, - {496: 2648, 725: 5166}, - {496: 2648, 725: 5165}, + {126: 5163, 5161, 5162, 5164, 839: 5160}, + {891: 5169}, + {497: 2649, 721: 5168}, + {497: 2649, 721: 5167}, + {497: 2649, 721: 5166}, // 2705 - {496: 2648, 725: 5164}, + {497: 2649, 721: 5165}, {159, 159}, {160, 160}, {161, 161}, @@ -9275,139 +9270,139 @@ var ( {213, 213, 13: 213, 46: 213, 463: 213}, {212, 212, 13: 212, 46: 212, 463: 212}, {211, 211, 13: 211, 46: 211, 463: 211}, - {210, 210, 13: 210, 46: 210, 131: 5176, 133: 5174, 5177, 5175, 463: 210, 493: 5182, 946: 5183}, + {210, 210, 13: 210, 46: 210, 131: 5177, 133: 5175, 5178, 5176, 463: 210, 494: 5183, 944: 5184}, // 2715 - {209, 209, 13: 209, 46: 209, 131: 209, 133: 209, 209, 209, 463: 209, 493: 209}, + {209, 209, 13: 209, 46: 209, 131: 209, 133: 209, 209, 209, 463: 209, 494: 209}, + {464: 5182}, {464: 5181}, {464: 5180}, {464: 5179}, - {464: 5178}, // 2720 - {203, 203, 13: 203, 46: 203, 131: 203, 133: 203, 203, 203, 463: 203, 493: 203}, - {204, 204, 13: 204, 46: 204, 131: 204, 133: 204, 204, 204, 463: 204, 493: 204}, - {205, 205, 13: 205, 46: 205, 131: 205, 133: 205, 205, 205, 463: 205, 493: 205}, - {206, 206, 13: 206, 46: 206, 131: 206, 133: 206, 206, 206, 463: 206, 493: 206}, - {131: 5176, 133: 5174, 5177, 5175, 946: 5184}, + {203, 203, 13: 203, 46: 203, 131: 203, 133: 203, 203, 203, 463: 203, 494: 203}, + {204, 204, 13: 204, 46: 204, 131: 204, 133: 204, 204, 204, 463: 204, 494: 204}, + {205, 205, 13: 205, 46: 205, 131: 205, 133: 205, 205, 205, 463: 205, 494: 205}, + {206, 206, 13: 206, 46: 206, 131: 206, 133: 206, 206, 206, 463: 206, 494: 206}, + {131: 5177, 133: 5175, 5178, 5176, 944: 5185}, // 2725 - {207, 207, 13: 207, 46: 207, 131: 207, 133: 207, 207, 207, 463: 207, 493: 207}, - {208, 208, 13: 208, 46: 208, 131: 208, 133: 208, 208, 208, 463: 208, 493: 208}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5186}, - {640: 5187}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 5188}, + {207, 207, 13: 207, 46: 207, 131: 207, 133: 207, 207, 207, 463: 207, 494: 207}, + {208, 208, 13: 208, 46: 208, 131: 208, 133: 208, 208, 208, 463: 208, 494: 208}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5187}, + {638: 5188}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 5189}, // 2730 - {164, 164, 7: 5094, 463: 5158, 1142: 5189}, + {164, 164, 7: 5095, 463: 5159, 1141: 5190}, {166, 166}, - {2134, 2134, 7: 2134, 14: 2134, 16: 2134, 21: 2134, 468: 2134, 473: 2134, 486: 2134, 489: 2134, 494: 2134, 510: 2134, 637: 2134, 640: 2134}, + {2135, 2135, 7: 2135, 14: 2135, 16: 2135, 19: 2135, 468: 2135, 473: 2135, 486: 2135, 489: 2135, 493: 2135, 495: 2135, 510: 2135, 635: 2135, 638: 2135}, {239, 239}, - {2: 826, 826, 826, 826, 826, 8: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 51: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 462: 826, 464: 826, 826, 826, 468: 826, 471: 826, 826, 474: 826, 826, 826, 479: 826, 481: 826, 484: 826, 486: 826, 491: 826, 494: 826, 826, 826, 503: 826, 512: 826, 532: 826, 555: 826, 826, 826, 826, 560: 826, 826, 826, 826, 826, 826, 567: 826, 826, 826, 826, 572: 826, 826, 575: 826, 577: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 638: 826, 641: 826, 735: 826, 826, 738: 826, 826, 826, 749: 826, 758: 826, 826, 826}, + {2: 826, 826, 826, 826, 826, 8: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 51: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 462: 826, 464: 826, 826, 826, 468: 826, 471: 826, 826, 474: 826, 826, 826, 479: 826, 481: 826, 484: 826, 486: 826, 491: 826, 495: 826, 826, 826, 503: 826, 512: 826, 534: 826, 555: 826, 826, 826, 826, 560: 826, 826, 826, 826, 826, 826, 567: 826, 826, 826, 826, 572: 826, 826, 575: 826, 577: 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 826, 636: 826, 639: 826, 733: 826, 826, 736: 826, 826, 826, 747: 826, 756: 826, 826, 826}, // 2735 - {2: 824, 824, 824, 824, 824, 8: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 51: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 462: 824, 479: 824, 484: 824, 486: 824, 568: 824, 738: 824, 824, 824}, - {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 462: 1027, 479: 1027, 568: 1027, 738: 5197, 5196, 5195, 825: 5198, 873: 5199}, - {2: 1030, 1030, 1030, 1030, 1030, 8: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 51: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 462: 1030, 464: 1030, 1030, 1030, 468: 1030, 471: 1030, 1030, 474: 1030, 1030, 1030, 479: 1030, 481: 1030, 484: 1030, 486: 1030, 491: 1030, 494: 1030, 1030, 1030, 503: 1030, 512: 1030, 532: 1030, 555: 1030, 1030, 1030, 1030, 560: 1030, 1030, 1030, 1030, 1030, 1030, 567: 1030, 1030, 1030, 1030, 572: 1030, 1030, 575: 1030, 577: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 638: 1030, 641: 1030, 735: 1030, 1030, 738: 1030, 1030, 1030, 749: 1030, 758: 1030, 1030, 1030}, - {2: 1029, 1029, 1029, 1029, 1029, 8: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 51: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 462: 1029, 464: 1029, 1029, 1029, 468: 1029, 471: 1029, 1029, 474: 1029, 1029, 1029, 479: 1029, 481: 1029, 484: 1029, 486: 1029, 491: 1029, 494: 1029, 1029, 1029, 503: 1029, 512: 1029, 532: 1029, 555: 1029, 1029, 1029, 1029, 560: 1029, 1029, 1029, 1029, 1029, 1029, 567: 1029, 1029, 1029, 1029, 572: 1029, 1029, 575: 1029, 577: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 638: 1029, 641: 1029, 735: 1029, 1029, 738: 1029, 1029, 1029, 749: 1029, 758: 1029, 1029, 1029}, - {2: 1028, 1028, 1028, 1028, 1028, 8: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 51: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 462: 1028, 464: 1028, 1028, 1028, 468: 1028, 471: 1028, 1028, 474: 1028, 1028, 1028, 479: 1028, 481: 1028, 484: 1028, 486: 1028, 491: 1028, 494: 1028, 1028, 1028, 503: 1028, 512: 1028, 532: 1028, 555: 1028, 1028, 1028, 1028, 560: 1028, 1028, 1028, 1028, 1028, 1028, 567: 1028, 1028, 1028, 1028, 572: 1028, 1028, 575: 1028, 577: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 638: 1028, 641: 1028, 735: 1028, 1028, 738: 1028, 1028, 1028, 749: 1028, 758: 1028, 1028, 1028}, + {2: 824, 824, 824, 824, 824, 8: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 51: 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 824, 462: 824, 479: 824, 484: 824, 486: 824, 568: 824, 736: 824, 824, 824}, + {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 462: 1027, 479: 1027, 568: 1027, 736: 5198, 5197, 5196, 823: 5199, 871: 5200}, + {2: 1030, 1030, 1030, 1030, 1030, 8: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 51: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 462: 1030, 464: 1030, 1030, 1030, 468: 1030, 471: 1030, 1030, 474: 1030, 1030, 1030, 479: 1030, 481: 1030, 484: 1030, 486: 1030, 491: 1030, 495: 1030, 1030, 1030, 503: 1030, 512: 1030, 534: 1030, 555: 1030, 1030, 1030, 1030, 560: 1030, 1030, 1030, 1030, 1030, 1030, 567: 1030, 1030, 1030, 1030, 572: 1030, 1030, 575: 1030, 577: 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 1030, 636: 1030, 639: 1030, 733: 1030, 1030, 736: 1030, 1030, 1030, 747: 1030, 756: 1030, 1030, 1030}, + {2: 1029, 1029, 1029, 1029, 1029, 8: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 51: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 462: 1029, 464: 1029, 1029, 1029, 468: 1029, 471: 1029, 1029, 474: 1029, 1029, 1029, 479: 1029, 481: 1029, 484: 1029, 486: 1029, 491: 1029, 495: 1029, 1029, 1029, 503: 1029, 512: 1029, 534: 1029, 555: 1029, 1029, 1029, 1029, 560: 1029, 1029, 1029, 1029, 1029, 1029, 567: 1029, 1029, 1029, 1029, 572: 1029, 1029, 575: 1029, 577: 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 1029, 636: 1029, 639: 1029, 733: 1029, 1029, 736: 1029, 1029, 1029, 747: 1029, 756: 1029, 1029, 1029}, + {2: 1028, 1028, 1028, 1028, 1028, 8: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 51: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 462: 1028, 464: 1028, 1028, 1028, 468: 1028, 471: 1028, 1028, 474: 1028, 1028, 1028, 479: 1028, 481: 1028, 484: 1028, 486: 1028, 491: 1028, 495: 1028, 1028, 1028, 503: 1028, 512: 1028, 534: 1028, 555: 1028, 1028, 1028, 1028, 560: 1028, 1028, 1028, 1028, 1028, 1028, 567: 1028, 1028, 1028, 1028, 572: 1028, 1028, 575: 1028, 577: 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 1028, 636: 1028, 639: 1028, 733: 1028, 1028, 736: 1028, 1028, 1028, 747: 1028, 756: 1028, 1028, 1028}, // 2740 {2: 1026, 1026, 1026, 1026, 1026, 8: 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 51: 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 1026, 462: 1026, 479: 1026, 484: 1026, 486: 1026, 568: 1026}, - {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 462: 1813, 479: 4085, 568: 1813, 842: 5200}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 568: 5204, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5203, 838: 5201, 876: 5202}, - {903, 903, 7: 903, 50: 903, 461: 903, 463: 903, 469: 903, 903, 477: 903, 903, 482: 903, 903, 903, 903, 488: 903, 903, 903, 501: 903, 903, 504: 903, 903}, - {7: 5255, 501: 5325}, + {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 462: 1813, 479: 4086, 568: 1813, 840: 5201}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 568: 5205, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5204, 836: 5202, 874: 5203}, + {903, 903, 7: 903, 50: 903, 461: 903, 463: 903, 469: 903, 903, 477: 903, 903, 482: 903, 903, 903, 903, 488: 903, 903, 903, 493: 903, 502: 903, 504: 903, 903}, + {7: 5256, 493: 5326}, // 2745 - {7: 901, 471: 5222, 5223, 501: 5312, 503: 5221, 506: 5224, 5220, 5225, 5226, 805: 5219, 811: 5218}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5309, 2674, 2675, 2673}, - {899, 899, 7: 899, 50: 899, 461: 899, 463: 899, 469: 899, 899, 899, 899, 477: 899, 899, 482: 899, 899, 899, 899, 488: 899, 899, 899, 501: 899, 899, 899, 899, 899, 899, 899, 899, 899, 511: 899}, - {898, 898, 7: 898, 50: 898, 461: 898, 463: 898, 469: 898, 898, 898, 898, 477: 898, 898, 482: 898, 898, 898, 898, 488: 898, 898, 898, 501: 898, 898, 898, 898, 898, 898, 898, 898, 898, 511: 898}, - {894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 463: 894, 467: 894, 469: 894, 894, 894, 894, 477: 894, 894, 894, 5259, 482: 894, 894, 894, 894, 488: 894, 894, 894, 492: 894, 501: 894, 894, 894, 894, 894, 894, 894, 894, 894, 511: 894, 514: 894, 894, 663: 894, 847: 5258}, + {7: 901, 471: 5223, 5224, 493: 5313, 503: 5222, 506: 5225, 5221, 5226, 5227, 803: 5220, 809: 5219}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5310, 2675, 2676, 2674}, + {899, 899, 7: 899, 50: 899, 461: 899, 463: 899, 469: 899, 899, 899, 899, 477: 899, 899, 482: 899, 899, 899, 899, 488: 899, 899, 899, 493: 899, 502: 899, 899, 899, 899, 899, 899, 899, 899, 511: 899}, + {898, 898, 7: 898, 50: 898, 461: 898, 463: 898, 469: 898, 898, 898, 898, 477: 898, 898, 482: 898, 898, 898, 898, 488: 898, 898, 898, 493: 898, 502: 898, 898, 898, 898, 898, 898, 898, 898, 511: 898}, + {894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 463: 894, 467: 894, 469: 894, 894, 894, 894, 477: 894, 894, 894, 5260, 482: 894, 894, 894, 894, 488: 894, 894, 894, 492: 894, 894, 502: 894, 894, 894, 894, 894, 894, 894, 894, 511: 894, 514: 894, 894, 661: 894, 845: 5259}, // 2750 - {892, 892, 2910, 2758, 2794, 2912, 2685, 892, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 892, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 892, 463: 892, 467: 5216, 469: 892, 892, 892, 892, 477: 892, 892, 482: 892, 892, 892, 892, 488: 892, 892, 892, 501: 892, 892, 892, 892, 892, 892, 892, 892, 892, 511: 892, 653: 5215, 2674, 2675, 2673, 902: 5214, 5213}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 2506, 491: 2505, 559: 2504, 568: 5204, 636: 2500, 653: 3818, 2674, 2675, 2673, 699: 5212, 727: 5207, 741: 3778, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 3780, 3779, 3777, 787: 5206, 790: 5205, 5211, 838: 5201, 876: 5210}, - {7: 5255, 50: 5256}, - {901, 901, 7: 901, 50: 901, 461: 901, 463: 901, 469: 901, 901, 5222, 5223, 477: 901, 901, 482: 901, 901, 901, 901, 488: 901, 901, 901, 501: 901, 901, 5221, 901, 901, 5224, 5220, 5225, 5226, 805: 5219, 811: 5218}, - {2: 2910, 2758, 2794, 2912, 2685, 892, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 3871, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 467: 5216, 469: 786, 471: 892, 892, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 503: 892, 506: 892, 892, 892, 892, 653: 5215, 2674, 2675, 2673, 756: 3788, 3789, 902: 5214, 5213}, + {892, 892, 2911, 2759, 2795, 2913, 2686, 892, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 892, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 892, 463: 892, 467: 5217, 469: 892, 892, 892, 892, 477: 892, 892, 482: 892, 892, 892, 892, 488: 892, 892, 892, 493: 892, 502: 892, 892, 892, 892, 892, 892, 892, 892, 511: 892, 651: 5216, 2675, 2676, 2674, 900: 5215, 5214}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 2507, 491: 2506, 559: 2505, 568: 5205, 634: 2501, 651: 3819, 2675, 2676, 2674, 697: 5213, 725: 5208, 739: 3779, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 3781, 3780, 3778, 785: 5207, 788: 5206, 5212, 836: 5202, 874: 5211}, + {7: 5256, 50: 5257}, + {901, 901, 7: 901, 50: 901, 461: 901, 463: 901, 469: 901, 901, 5223, 5224, 477: 901, 901, 482: 901, 901, 901, 901, 488: 901, 901, 901, 493: 901, 502: 901, 5222, 901, 901, 5225, 5221, 5226, 5227, 803: 5220, 809: 5219}, + {2: 2911, 2759, 2795, 2913, 2686, 892, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 3872, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 467: 5217, 469: 786, 471: 892, 892, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 503: 892, 506: 892, 892, 892, 892, 651: 5216, 2675, 2676, 2674, 754: 3789, 3790, 900: 5215, 5214}, // 2755 - {896, 896, 7: 896, 50: 896, 461: 896, 463: 896, 469: 896, 896, 896, 896, 477: 896, 896, 482: 896, 896, 896, 896, 488: 896, 896, 896, 501: 896, 896, 896, 896, 896, 896, 896, 896, 896, 511: 896}, - {891, 891, 7: 891, 50: 891, 461: 891, 463: 891, 469: 891, 891, 891, 891, 477: 891, 891, 891, 482: 891, 891, 891, 891, 488: 891, 891, 891, 492: 891, 501: 891, 891, 891, 891, 891, 891, 891, 891, 891, 511: 891, 514: 891, 891, 663: 891}, - {890, 890, 7: 890, 50: 890, 461: 890, 463: 890, 469: 890, 890, 890, 890, 477: 890, 890, 890, 482: 890, 890, 890, 890, 488: 890, 890, 890, 492: 890, 501: 890, 890, 890, 890, 890, 890, 890, 890, 890, 511: 890, 514: 890, 890, 663: 890}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5217, 2674, 2675, 2673}, - {889, 889, 7: 889, 50: 889, 461: 889, 463: 889, 469: 889, 889, 889, 889, 477: 889, 889, 889, 482: 889, 889, 889, 889, 488: 889, 889, 889, 492: 889, 501: 889, 889, 889, 889, 889, 889, 889, 889, 889, 511: 889, 514: 889, 889, 663: 889}, + {896, 896, 7: 896, 50: 896, 461: 896, 463: 896, 469: 896, 896, 896, 896, 477: 896, 896, 482: 896, 896, 896, 896, 488: 896, 896, 896, 493: 896, 502: 896, 896, 896, 896, 896, 896, 896, 896, 511: 896}, + {891, 891, 7: 891, 50: 891, 461: 891, 463: 891, 469: 891, 891, 891, 891, 477: 891, 891, 891, 482: 891, 891, 891, 891, 488: 891, 891, 891, 492: 891, 891, 502: 891, 891, 891, 891, 891, 891, 891, 891, 511: 891, 514: 891, 891, 661: 891}, + {890, 890, 7: 890, 50: 890, 461: 890, 463: 890, 469: 890, 890, 890, 890, 477: 890, 890, 890, 482: 890, 890, 890, 890, 488: 890, 890, 890, 492: 890, 890, 502: 890, 890, 890, 890, 890, 890, 890, 890, 511: 890, 514: 890, 890, 661: 890}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5218, 2675, 2676, 2674}, + {889, 889, 7: 889, 50: 889, 461: 889, 463: 889, 469: 889, 889, 889, 889, 477: 889, 889, 889, 482: 889, 889, 889, 889, 488: 889, 889, 889, 492: 889, 889, 502: 889, 889, 889, 889, 889, 889, 889, 889, 511: 889, 514: 889, 889, 661: 889}, // 2760 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5248}, - {506: 860, 895: 5235, 1077: 5239}, - {471: 5222, 5223, 506: 5232, 805: 5233}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5229}, - {506: 862, 895: 862}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5249}, + {506: 860, 893: 5236, 1076: 5240}, + {471: 5223, 5224, 506: 5233, 803: 5234}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5230}, + {506: 862, 893: 862}, // 2765 - {506: 861, 895: 861}, + {506: 861, 893: 861}, {2: 858, 858, 858, 858, 858, 8: 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 51: 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 858, 462: 858}, + {506: 5229}, {506: 5228}, - {506: 5227}, {2: 856, 856, 856, 856, 856, 8: 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 51: 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 856, 462: 856}, // 2770 {2: 857, 857, 857, 857, 857, 8: 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 51: 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 857, 462: 857}, - {864, 864, 7: 864, 50: 864, 461: 5230, 463: 864, 469: 864, 864, 864, 864, 477: 864, 864, 482: 864, 864, 864, 864, 488: 864, 864, 864, 501: 864, 864, 864, 864, 864, 864, 864, 864, 864, 511: 864, 805: 5219, 811: 5218}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5231}, - {863, 863, 7: 863, 50: 863, 461: 863, 463: 863, 469: 863, 863, 863, 863, 477: 863, 863, 482: 863, 863, 863, 863, 488: 863, 863, 863, 493: 3247, 497: 3245, 3246, 3244, 3242, 863, 863, 863, 863, 863, 863, 863, 863, 863, 511: 863, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5238}, + {864, 864, 7: 864, 50: 864, 461: 5231, 463: 864, 469: 864, 864, 864, 864, 477: 864, 864, 482: 864, 864, 864, 864, 488: 864, 864, 864, 493: 864, 502: 864, 864, 864, 864, 864, 864, 864, 864, 511: 864, 803: 5220, 809: 5219}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5232}, + {863, 863, 7: 863, 50: 863, 461: 863, 463: 863, 469: 863, 863, 863, 863, 477: 863, 863, 482: 863, 863, 863, 863, 488: 863, 863, 863, 493: 863, 3246, 498: 3244, 3245, 3243, 3241, 863, 863, 863, 863, 863, 863, 863, 863, 511: 863, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5239}, // 2775 - {506: 860, 895: 5235, 1077: 5234}, - {506: 5236}, + {506: 860, 893: 5236, 1076: 5235}, + {506: 5237}, {506: 859}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5237}, - {865, 865, 7: 865, 50: 865, 461: 865, 463: 865, 469: 865, 865, 865, 865, 477: 865, 865, 482: 865, 865, 865, 865, 488: 865, 865, 865, 501: 865, 865, 865, 865, 865, 865, 865, 865, 865, 511: 865, 805: 5219, 811: 5218}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5238}, + {865, 865, 7: 865, 50: 865, 461: 865, 463: 865, 469: 865, 865, 865, 865, 477: 865, 865, 482: 865, 865, 865, 865, 488: 865, 865, 865, 493: 865, 502: 865, 865, 865, 865, 865, 865, 865, 865, 511: 865, 803: 5220, 809: 5219}, // 2780 - {866, 866, 7: 866, 50: 866, 461: 866, 463: 866, 469: 866, 866, 866, 866, 477: 866, 866, 482: 866, 866, 866, 866, 488: 866, 866, 866, 501: 866, 866, 866, 866, 866, 866, 866, 866, 866, 511: 866, 805: 5219, 811: 5218}, - {506: 5240}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5241}, - {461: 5242, 470: 5243, 5222, 5223, 503: 5221, 506: 5224, 5220, 5225, 5226, 805: 5219, 811: 5218}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5247}, + {866, 866, 7: 866, 50: 866, 461: 866, 463: 866, 469: 866, 866, 866, 866, 477: 866, 866, 482: 866, 866, 866, 866, 488: 866, 866, 866, 493: 866, 502: 866, 866, 866, 866, 866, 866, 866, 866, 511: 866, 803: 5220, 809: 5219}, + {506: 5241}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5242}, + {461: 5243, 470: 5244, 5223, 5224, 503: 5222, 506: 5225, 5221, 5226, 5227, 803: 5220, 809: 5219}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5248}, // 2785 - {462: 5244}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 5245}, - {7: 4041, 50: 5246}, - {867, 867, 7: 867, 50: 867, 461: 867, 463: 867, 469: 867, 867, 867, 867, 477: 867, 867, 482: 867, 867, 867, 867, 488: 867, 867, 867, 501: 867, 867, 867, 867, 867, 867, 867, 867, 867, 511: 867}, - {868, 868, 7: 868, 50: 868, 461: 868, 463: 868, 469: 868, 868, 868, 868, 477: 868, 868, 482: 868, 868, 868, 868, 488: 868, 868, 868, 493: 3247, 497: 3245, 3246, 3244, 3242, 868, 868, 868, 868, 868, 868, 868, 868, 868, 511: 868, 723: 3243, 3241}, + {462: 5245}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 5246}, + {7: 4042, 50: 5247}, + {867, 867, 7: 867, 50: 867, 461: 867, 463: 867, 469: 867, 867, 867, 867, 477: 867, 867, 482: 867, 867, 867, 867, 488: 867, 867, 867, 493: 867, 502: 867, 867, 867, 867, 867, 867, 867, 867, 511: 867}, + {868, 868, 7: 868, 50: 868, 461: 868, 463: 868, 469: 868, 868, 868, 868, 477: 868, 868, 482: 868, 868, 868, 868, 488: 868, 868, 868, 493: 868, 3246, 498: 3244, 3245, 3243, 3241, 868, 868, 868, 868, 868, 868, 868, 868, 511: 868, 722: 3242, 3240}, // 2790 - {871, 871, 7: 871, 50: 871, 461: 5249, 463: 871, 469: 871, 5250, 5222, 5223, 477: 871, 871, 482: 871, 871, 871, 871, 488: 871, 871, 871, 501: 871, 871, 5221, 871, 871, 5224, 5220, 5225, 5226, 511: 871, 805: 5219, 811: 5218}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5254}, - {462: 5251}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 5252}, - {7: 4041, 50: 5253}, + {871, 871, 7: 871, 50: 871, 461: 5250, 463: 871, 469: 871, 5251, 5223, 5224, 477: 871, 871, 482: 871, 871, 871, 871, 488: 871, 871, 871, 493: 871, 502: 871, 5222, 871, 871, 5225, 5221, 5226, 5227, 511: 871, 803: 5220, 809: 5219}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5255}, + {462: 5252}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 5253}, + {7: 4042, 50: 5254}, // 2795 - {869, 869, 7: 869, 50: 869, 461: 869, 463: 869, 469: 869, 869, 869, 869, 477: 869, 869, 482: 869, 869, 869, 869, 488: 869, 869, 869, 501: 869, 869, 869, 869, 869, 869, 869, 869, 869, 511: 869}, - {870, 870, 7: 870, 50: 870, 461: 870, 463: 870, 469: 870, 870, 870, 870, 477: 870, 870, 482: 870, 870, 870, 870, 488: 870, 870, 870, 493: 3247, 497: 3245, 3246, 3244, 3242, 870, 870, 870, 870, 870, 870, 870, 870, 870, 511: 870, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 568: 5204, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5211, 838: 5257}, - {895, 895, 7: 895, 50: 895, 461: 895, 463: 895, 469: 895, 895, 895, 895, 477: 895, 895, 482: 895, 895, 895, 895, 488: 895, 895, 895, 501: 895, 895, 895, 895, 895, 895, 895, 895, 895, 511: 895}, - {902, 902, 7: 902, 50: 902, 461: 902, 463: 902, 469: 902, 902, 477: 902, 902, 482: 902, 902, 902, 902, 488: 902, 902, 902, 501: 902, 902, 504: 902, 902}, + {869, 869, 7: 869, 50: 869, 461: 869, 463: 869, 469: 869, 869, 869, 869, 477: 869, 869, 482: 869, 869, 869, 869, 488: 869, 869, 869, 493: 869, 502: 869, 869, 869, 869, 869, 869, 869, 869, 511: 869}, + {870, 870, 7: 870, 50: 870, 461: 870, 463: 870, 469: 870, 870, 870, 870, 477: 870, 870, 482: 870, 870, 870, 870, 488: 870, 870, 870, 493: 870, 3246, 498: 3244, 3245, 3243, 3241, 870, 870, 870, 870, 870, 870, 870, 870, 511: 870, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 568: 5205, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5212, 836: 5258}, + {895, 895, 7: 895, 50: 895, 461: 895, 463: 895, 469: 895, 895, 895, 895, 477: 895, 895, 482: 895, 895, 895, 895, 488: 895, 895, 895, 493: 895, 502: 895, 895, 895, 895, 895, 895, 895, 895, 511: 895}, + {902, 902, 7: 902, 50: 902, 461: 902, 463: 902, 469: 902, 902, 477: 902, 902, 482: 902, 902, 902, 902, 488: 902, 902, 902, 493: 902, 502: 902, 504: 902, 902}, // 2800 - {892, 892, 2910, 2758, 2794, 2912, 2685, 892, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 892, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 892, 463: 892, 467: 5216, 469: 892, 892, 892, 892, 477: 892, 892, 892, 482: 892, 892, 892, 892, 488: 892, 892, 892, 492: 892, 501: 892, 892, 892, 892, 892, 892, 892, 892, 892, 511: 892, 514: 892, 892, 653: 5215, 2674, 2675, 2673, 663: 892, 902: 5214, 5263}, - {462: 5260}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 5261}, - {7: 4627, 50: 5262}, - {893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 467: 893, 469: 893, 893, 893, 893, 477: 893, 893, 893, 482: 893, 893, 893, 893, 488: 893, 893, 893, 893, 893, 501: 893, 893, 893, 893, 893, 893, 893, 893, 893, 511: 893, 514: 893, 893, 544: 893, 559: 893, 636: 893, 639: 893, 649: 893, 663: 893}, + {892, 892, 2911, 2759, 2795, 2913, 2686, 892, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 892, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 892, 463: 892, 467: 5217, 469: 892, 892, 892, 892, 477: 892, 892, 892, 482: 892, 892, 892, 892, 488: 892, 892, 892, 492: 892, 892, 502: 892, 892, 892, 892, 892, 892, 892, 892, 511: 892, 514: 892, 892, 651: 5216, 2675, 2676, 2674, 661: 892, 900: 5215, 5264}, + {462: 5261}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 5262}, + {7: 4628, 50: 5263}, + {893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 893, 467: 893, 469: 893, 893, 893, 893, 477: 893, 893, 893, 482: 893, 893, 893, 893, 488: 893, 893, 893, 893, 893, 893, 502: 893, 893, 893, 893, 893, 893, 893, 893, 511: 893, 514: 893, 893, 544: 893, 559: 893, 634: 893, 637: 893, 647: 893, 661: 893}, // 2805 - {1820, 1820, 7: 1820, 50: 1820, 461: 1820, 463: 1820, 469: 1820, 1820, 1820, 1820, 477: 1820, 1820, 1820, 482: 1820, 1820, 1820, 1820, 488: 1820, 1820, 1820, 492: 1820, 501: 1820, 1820, 1820, 1820, 1820, 1820, 1820, 1820, 1820, 511: 1820, 514: 1820, 1820, 663: 5265, 911: 5264, 1153: 5266}, - {1819, 1819, 7: 1819, 50: 1819, 461: 1819, 463: 1819, 469: 1819, 1819, 1819, 1819, 477: 1819, 1819, 1819, 482: 1819, 1819, 1819, 1819, 488: 1819, 1819, 1819, 492: 1819, 501: 1819, 1819, 1819, 1819, 1819, 1819, 1819, 1819, 1819, 511: 1819, 514: 1819, 1819}, - {214: 5307}, - {873, 873, 7: 873, 50: 873, 461: 873, 463: 873, 469: 873, 873, 873, 873, 477: 873, 873, 5269, 482: 873, 873, 873, 873, 488: 873, 873, 873, 492: 5270, 501: 873, 873, 873, 873, 873, 873, 873, 873, 873, 511: 873, 514: 5268, 873, 927: 5272, 5271, 1043: 5273, 5267}, - {988, 988, 7: 988, 50: 988, 461: 988, 463: 988, 469: 988, 988, 988, 988, 477: 988, 988, 482: 988, 988, 988, 988, 488: 988, 988, 988, 501: 988, 988, 988, 988, 988, 988, 988, 988, 988, 511: 988, 515: 5288, 1297: 5289}, + {1820, 1820, 7: 1820, 50: 1820, 461: 1820, 463: 1820, 469: 1820, 1820, 1820, 1820, 477: 1820, 1820, 1820, 482: 1820, 1820, 1820, 1820, 488: 1820, 1820, 1820, 492: 1820, 1820, 502: 1820, 1820, 1820, 1820, 1820, 1820, 1820, 1820, 511: 1820, 514: 1820, 1820, 661: 5266, 909: 5265, 1152: 5267}, + {1819, 1819, 7: 1819, 50: 1819, 461: 1819, 463: 1819, 469: 1819, 1819, 1819, 1819, 477: 1819, 1819, 1819, 482: 1819, 1819, 1819, 1819, 488: 1819, 1819, 1819, 492: 1819, 1819, 502: 1819, 1819, 1819, 1819, 1819, 1819, 1819, 1819, 511: 1819, 514: 1819, 1819}, + {214: 5308}, + {873, 873, 7: 873, 50: 873, 461: 873, 463: 873, 469: 873, 873, 873, 873, 477: 873, 873, 5270, 482: 873, 873, 873, 873, 488: 873, 873, 873, 492: 5271, 873, 502: 873, 873, 873, 873, 873, 873, 873, 873, 511: 873, 514: 5269, 873, 925: 5273, 5272, 1041: 5274, 5268}, + {988, 988, 7: 988, 50: 988, 461: 988, 463: 988, 469: 988, 988, 988, 988, 477: 988, 988, 482: 988, 988, 988, 988, 488: 988, 988, 988, 493: 988, 502: 988, 988, 988, 988, 988, 988, 988, 988, 511: 988, 515: 5289, 1295: 5290}, // 2810 - {566: 4300, 639: 4301, 815: 5287}, - {566: 4300, 639: 4301, 815: 5286}, - {566: 4300, 639: 4301, 815: 5285}, - {462: 885, 482: 5275, 1207: 5276}, - {875, 875, 7: 875, 50: 875, 461: 875, 463: 875, 469: 875, 875, 875, 875, 477: 875, 875, 875, 482: 875, 875, 875, 875, 488: 875, 875, 875, 492: 875, 501: 875, 875, 875, 875, 875, 875, 875, 875, 875, 511: 875, 514: 875, 875}, + {566: 4301, 637: 4302, 813: 5288}, + {566: 4301, 637: 4302, 813: 5287}, + {566: 4301, 637: 4302, 813: 5286}, + {462: 885, 482: 5276, 1206: 5277}, + {875, 875, 7: 875, 50: 875, 461: 875, 463: 875, 469: 875, 875, 875, 875, 477: 875, 875, 875, 482: 875, 875, 875, 875, 488: 875, 875, 875, 492: 875, 875, 502: 875, 875, 875, 875, 875, 875, 875, 875, 511: 875, 514: 875, 875}, // 2815 - {872, 872, 7: 872, 50: 872, 461: 872, 463: 872, 469: 872, 872, 872, 872, 477: 872, 872, 5269, 482: 872, 872, 872, 872, 488: 872, 872, 872, 492: 5270, 501: 872, 872, 872, 872, 872, 872, 872, 872, 872, 511: 872, 514: 5268, 872, 927: 5274, 5271}, - {874, 874, 7: 874, 50: 874, 461: 874, 463: 874, 469: 874, 874, 874, 874, 477: 874, 874, 874, 482: 874, 874, 874, 874, 488: 874, 874, 874, 492: 874, 501: 874, 874, 874, 874, 874, 874, 874, 874, 874, 511: 874, 514: 874, 874}, - {490: 5281, 502: 5282, 506: 5280}, - {462: 5277}, - {2: 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 880, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 5278}, + {872, 872, 7: 872, 50: 872, 461: 872, 463: 872, 469: 872, 872, 872, 872, 477: 872, 872, 5270, 482: 872, 872, 872, 872, 488: 872, 872, 872, 492: 5271, 872, 502: 872, 872, 872, 872, 872, 872, 872, 872, 511: 872, 514: 5269, 872, 925: 5275, 5272}, + {874, 874, 7: 874, 50: 874, 461: 874, 463: 874, 469: 874, 874, 874, 874, 477: 874, 874, 874, 482: 874, 874, 874, 874, 488: 874, 874, 874, 492: 874, 874, 502: 874, 874, 874, 874, 874, 874, 874, 874, 511: 874, 514: 874, 874}, + {490: 5282, 502: 5283, 506: 5281}, + {462: 5278}, + {2: 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 880, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 5279}, // 2820 - {7: 4886, 50: 5279}, - {881, 881, 7: 881, 50: 881, 461: 881, 463: 881, 469: 881, 881, 881, 881, 477: 881, 881, 881, 482: 881, 881, 881, 881, 488: 881, 881, 881, 492: 881, 501: 881, 881, 881, 881, 881, 881, 881, 881, 881, 511: 881, 514: 881, 881}, + {7: 4887, 50: 5280}, + {881, 881, 7: 881, 50: 881, 461: 881, 463: 881, 469: 881, 881, 881, 881, 477: 881, 881, 881, 482: 881, 881, 881, 881, 488: 881, 881, 881, 492: 881, 881, 502: 881, 881, 881, 881, 881, 881, 881, 881, 511: 881, 514: 881, 881}, {462: 884}, - {649: 5284}, - {649: 5283}, + {647: 5285}, + {647: 5284}, // 2825 {462: 882}, {462: 883}, @@ -9415,69 +9410,69 @@ var ( {462: 887, 482: 887}, {462: 888, 482: 888}, // 2830 - {86: 5293, 297: 5292, 378: 5291, 462: 985, 1296: 5290}, - {897, 897, 7: 897, 50: 897, 461: 897, 463: 897, 469: 897, 897, 897, 897, 477: 897, 897, 482: 897, 897, 897, 897, 488: 897, 897, 897, 501: 897, 897, 897, 897, 897, 897, 897, 897, 897, 511: 897}, - {462: 5294}, + {86: 5294, 299: 5293, 379: 5292, 462: 985, 1294: 5291}, + {897, 897, 7: 897, 50: 897, 461: 897, 463: 897, 469: 897, 897, 897, 897, 477: 897, 897, 482: 897, 897, 897, 897, 488: 897, 897, 897, 493: 897, 502: 897, 897, 897, 897, 897, 897, 897, 897, 511: 897}, + {462: 5295}, {462: 984}, {462: 983}, // 2835 {462: 982}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 5296, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5295}, - {50: 981, 357: 5304, 493: 3247, 497: 3245, 3246, 3244, 3242, 513: 5303, 723: 3243, 3241, 1298: 5302}, - {978, 978, 7: 978, 50: 978, 210: 5298, 461: 978, 463: 978, 469: 978, 978, 978, 978, 477: 978, 978, 482: 978, 978, 978, 978, 488: 978, 978, 978, 501: 978, 978, 978, 978, 978, 978, 978, 978, 978, 511: 978, 1095: 5297}, - {986, 986, 7: 986, 50: 986, 461: 986, 463: 986, 469: 986, 986, 986, 986, 477: 986, 986, 482: 986, 986, 986, 986, 488: 986, 986, 986, 501: 986, 986, 986, 986, 986, 986, 986, 986, 986, 511: 986}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 5297, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5296}, + {50: 981, 359: 5305, 494: 3246, 498: 3244, 3245, 3243, 3241, 513: 5304, 722: 3242, 3240, 1296: 5303}, + {978, 978, 7: 978, 50: 978, 210: 5299, 461: 978, 463: 978, 469: 978, 978, 978, 978, 477: 978, 978, 482: 978, 978, 978, 978, 488: 978, 978, 978, 493: 978, 502: 978, 978, 978, 978, 978, 978, 978, 978, 511: 978, 1094: 5298}, + {986, 986, 7: 986, 50: 986, 461: 986, 463: 986, 469: 986, 986, 986, 986, 477: 986, 986, 482: 986, 986, 986, 986, 488: 986, 986, 986, 493: 986, 502: 986, 986, 986, 986, 986, 986, 986, 986, 511: 986}, // 2840 - {462: 5299}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5300}, - {50: 5301, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {977, 977, 7: 977, 50: 977, 461: 977, 463: 977, 469: 977, 977, 977, 977, 477: 977, 977, 482: 977, 977, 977, 977, 488: 977, 977, 977, 501: 977, 977, 977, 977, 977, 977, 977, 977, 977, 511: 977}, - {50: 5305}, + {462: 5300}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5301}, + {50: 5302, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {977, 977, 7: 977, 50: 977, 461: 977, 463: 977, 469: 977, 977, 977, 977, 477: 977, 977, 482: 977, 977, 977, 977, 488: 977, 977, 977, 493: 977, 502: 977, 977, 977, 977, 977, 977, 977, 977, 511: 977}, + {50: 5306}, // 2845 {50: 980}, {50: 979}, - {978, 978, 7: 978, 50: 978, 210: 5298, 461: 978, 463: 978, 469: 978, 978, 978, 978, 477: 978, 978, 482: 978, 978, 978, 978, 488: 978, 978, 978, 501: 978, 978, 978, 978, 978, 978, 978, 978, 978, 511: 978, 1095: 5306}, - {987, 987, 7: 987, 50: 987, 461: 987, 463: 987, 469: 987, 987, 987, 987, 477: 987, 987, 482: 987, 987, 987, 987, 488: 987, 987, 987, 501: 987, 987, 987, 987, 987, 987, 987, 987, 987, 511: 987}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5308}, + {978, 978, 7: 978, 50: 978, 210: 5299, 461: 978, 463: 978, 469: 978, 978, 978, 978, 477: 978, 978, 482: 978, 978, 978, 978, 488: 978, 978, 978, 493: 978, 502: 978, 978, 978, 978, 978, 978, 978, 978, 511: 978, 1094: 5307}, + {987, 987, 7: 987, 50: 987, 461: 987, 463: 987, 469: 987, 987, 987, 987, 477: 987, 987, 482: 987, 987, 987, 987, 488: 987, 987, 987, 493: 987, 502: 987, 987, 987, 987, 987, 987, 987, 987, 511: 987}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5309}, // 2850 - {1818, 1818, 7: 1818, 50: 1818, 461: 1818, 463: 1818, 469: 1818, 1818, 1818, 1818, 477: 1818, 1818, 1818, 482: 1818, 1818, 1818, 1818, 488: 1818, 1818, 1818, 492: 1818, 3247, 497: 3245, 3246, 3244, 3242, 1818, 1818, 1818, 1818, 1818, 1818, 1818, 1818, 1818, 511: 1818, 514: 1818, 1818, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5310}, - {471: 5222, 5223, 503: 5221, 506: 5224, 5220, 5225, 5226, 511: 5311, 805: 5219, 811: 5218}, - {900, 900, 7: 900, 50: 900, 461: 900, 463: 900, 469: 900, 900, 477: 900, 900, 482: 900, 900, 900, 900, 488: 900, 900, 900, 501: 900, 900, 504: 900, 900}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 5313, 878: 5314, 912: 5315}, + {1818, 1818, 7: 1818, 50: 1818, 461: 1818, 463: 1818, 469: 1818, 1818, 1818, 1818, 477: 1818, 1818, 1818, 482: 1818, 1818, 1818, 1818, 488: 1818, 1818, 1818, 492: 1818, 1818, 3246, 498: 3244, 3245, 3243, 3241, 1818, 1818, 1818, 1818, 1818, 1818, 1818, 1818, 511: 1818, 514: 1818, 1818, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5311}, + {471: 5223, 5224, 503: 5222, 506: 5225, 5221, 5226, 5227, 511: 5312, 803: 5220, 809: 5219}, + {900, 900, 7: 900, 50: 900, 461: 900, 463: 900, 469: 900, 900, 477: 900, 900, 482: 900, 900, 900, 900, 488: 900, 900, 900, 493: 900, 502: 900, 504: 900, 900}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 5314, 876: 5315, 910: 5316}, // 2855 - {487: 5323}, - {2285, 2285, 7: 2285, 470: 2285, 483: 2285, 489: 2285, 2285}, - {237, 237, 7: 5316, 470: 237, 483: 237, 489: 2634, 237, 778: 2635, 5317}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 5313, 878: 5322}, - {1265, 1265, 470: 1265, 483: 1265, 490: 2637, 756: 2638, 799: 5318}, + {487: 5324}, + {2286, 2286, 7: 2286, 470: 2286, 483: 2286, 489: 2286, 2286}, + {237, 237, 7: 5317, 470: 237, 483: 237, 489: 2635, 237, 776: 2636, 5318}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 5314, 876: 5323}, + {1265, 1265, 470: 1265, 483: 1265, 490: 2638, 754: 2639, 797: 5319}, // 2860 - {855, 855, 470: 855, 483: 5319, 1052: 5320}, - {496: 2648, 567: 2650, 725: 2647, 734: 2649, 868: 5321}, + {855, 855, 470: 855, 483: 5320, 1050: 5321}, + {497: 2649, 567: 2651, 721: 2648, 731: 2650, 866: 5322}, {241, 241, 470: 241}, {854, 854, 470: 854}, - {2284, 2284, 7: 2284, 470: 2284, 483: 2284, 489: 2284, 2284}, + {2285, 2285, 7: 2285, 470: 2285, 483: 2285, 489: 2285, 2285}, // 2865 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5324}, - {2286, 2286, 7: 2286, 470: 2286, 483: 2286, 489: 2286, 2286}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 5313, 878: 5314, 912: 5326}, - {237, 237, 7: 5316, 470: 237, 489: 2634, 778: 2635, 5327}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5325}, + {2287, 2287, 7: 2287, 470: 2287, 483: 2287, 489: 2287, 2287}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 5314, 876: 5315, 910: 5327}, + {237, 237, 7: 5317, 470: 237, 489: 2635, 776: 2636, 5328}, {240, 240, 470: 240}, // 2870 {2: 379, 379, 379, 379, 379, 8: 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 51: 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379, 379}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5330}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5331}, {378, 378}, - {19: 5343, 110: 5333, 125: 5336, 142: 556, 181: 5335, 188: 5346, 197: 5344, 213: 5337, 224: 5341, 244: 5345, 247: 5338, 532: 5342, 559: 5332, 1127: 5340, 1195: 5334, 1225: 5339}, + {20: 5344, 110: 5334, 125: 5337, 142: 556, 182: 5336, 189: 5347, 198: 5345, 213: 5338, 224: 5342, 244: 5346, 247: 5339, 534: 5343, 559: 5333, 1126: 5341, 1194: 5335, 1223: 5340}, {2001, 2001, 2001, 2001, 2001, 2001, 2001, 8: 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 51: 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 2001, 463: 2001, 555: 2001}, // 2875 {2000, 2000, 2000, 2000, 2000, 2000, 2000, 8: 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 51: 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 2000, 463: 2000, 555: 2000}, {566, 566}, {563, 563}, {562, 562}, - {206: 5353}, + {206: 5354}, // 2880 {560, 560}, - {142: 5352}, - {547, 547, 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 547, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 4193, 1126: 5347}, + {142: 5353}, + {547, 547, 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 547, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 4194, 1125: 5348}, {557, 557}, {142: 555}, // 2885 @@ -9485,457 +9480,457 @@ var ( {142: 553}, {142: 552}, {142: 551}, - {543, 543, 463: 5349, 1325: 5348}, + {543, 543, 463: 5350, 1323: 5349}, // 2890 {558, 558}, - {661: 5350}, - {485: 5351}, + {659: 5351}, + {485: 5352}, {542, 542}, {559, 559}, // 2895 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5354, 2674, 2675, 2673, 939: 5355}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5355, 2675, 2676, 2674, 937: 5356}, {565, 565, 7: 565}, - {561, 561, 7: 5356}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5357, 2674, 2675, 2673}, + {561, 561, 7: 5357}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5358, 2675, 2676, 2674}, {564, 564, 7: 564}, // 2900 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 5461, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 5462, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 5463, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5464}, - {559: 5447, 639: 5448}, - {639: 5444}, - {559: 5439, 639: 5438}, - {559: 5436}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 5462, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 5463, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 5464, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5465}, + {559: 5448, 637: 5449}, + {637: 5445}, + {559: 5440, 637: 5439}, + {559: 5437}, // 2905 - {314: 5430}, - {138: 5427, 211: 5429, 323: 5425, 352: 5426, 901: 5428}, - {193: 5422, 196: 5421}, - {559: 5380}, - {138: 5374, 156: 5376, 165: 575, 187: 5378, 249: 5377, 1283: 5375}, + {316: 5431}, + {138: 5428, 211: 5430, 325: 5426, 354: 5427, 899: 5429}, + {194: 5423, 197: 5422}, + {559: 5381}, + {138: 5375, 156: 5377, 165: 575, 188: 5379, 249: 5378, 1281: 5376}, // 2910 + {138: 5374}, {138: 5373}, - {138: 5372}, - {381: 5371}, + {382: 5372}, {675, 675}, {680, 680}, // 2915 {681, 681}, {682, 682}, - {165: 5379}, + {165: 5380}, {165: 574}, {165: 573}, // 2920 {165: 572}, {674, 674}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5381}, - {664: 5382, 918: 5383}, - {156: 5386, 160: 5385, 559: 2016, 934: 5384}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5382}, + {662: 5383, 916: 5384}, + {156: 5387, 160: 5386, 559: 2016, 932: 5385}, // 2925 {683, 683}, - {559: 5388}, + {559: 5389}, {110: 2015, 559: 2015}, - {160: 5387}, + {160: 5388}, {110: 2014, 559: 2014}, // 2930 - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 5389}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5390}, - {425, 425, 4: 425, 425, 425, 13: 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 461: 425, 5394, 425, 467: 425, 425, 425, 473: 425, 479: 425, 425, 491: 425, 425, 494: 425, 425, 510: 5393, 559: 425, 636: 425, 425, 639: 425, 1219: 5392, 1293: 5391}, - {385, 385, 4: 4132, 4134, 389, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 461: 385, 385, 385, 467: 385, 4131, 4168, 473: 2118, 479: 385, 385, 491: 385, 4845, 494: 2118, 385, 559: 385, 636: 385, 2118, 639: 4137, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4170, 906: 5409, 1001: 5408}, - {2121, 2121, 461: 5402, 1067: 5401}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 5390}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5391}, + {425, 425, 4: 425, 425, 425, 13: 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 425, 461: 425, 5395, 425, 467: 425, 425, 425, 473: 425, 479: 425, 425, 491: 425, 425, 495: 425, 425, 510: 5394, 559: 425, 634: 425, 425, 637: 425, 1218: 5393, 1291: 5392}, + {385, 385, 4: 4133, 4135, 389, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 461: 385, 385, 385, 467: 385, 4132, 4169, 473: 2118, 479: 385, 385, 491: 385, 4846, 495: 2118, 385, 559: 385, 634: 385, 2118, 637: 4138, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4171, 904: 5410, 999: 5409}, + {2121, 2121, 461: 5403, 1066: 5402}, // 2935 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5400}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 510: 5395, 566: 2335, 574: 2335, 576: 2335, 632: 2335, 4479, 639: 2335, 653: 4038, 2674, 2675, 2673, 665: 2335, 2335, 733: 4346, 820: 4688, 834: 4820, 890: 4821, 952: 4822, 1124: 5396}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5398}, - {7: 4824, 50: 5397}, - {424, 424, 4: 424, 424, 424, 13: 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 461: 424, 424, 424, 467: 424, 424, 424, 473: 424, 479: 424, 424, 491: 424, 424, 494: 424, 424, 559: 424, 636: 424, 424, 639: 424}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5401}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 510: 5396, 566: 2336, 574: 2336, 576: 2336, 630: 2336, 4480, 637: 2336, 651: 4039, 2675, 2676, 2674, 663: 2336, 2336, 732: 4347, 818: 4689, 832: 4821, 888: 4822, 950: 4823, 1123: 5397}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5399}, + {7: 4825, 50: 5398}, + {424, 424, 4: 424, 424, 424, 13: 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 424, 461: 424, 424, 424, 467: 424, 424, 424, 473: 424, 479: 424, 424, 491: 424, 424, 495: 424, 424, 559: 424, 634: 424, 424, 637: 424}, // 2940 - {50: 5399}, + {50: 5400}, {2049, 2049, 461: 2049}, {2050, 2050, 461: 2050}, {2122, 2122}, - {145: 5403}, + {145: 5404}, // 2945 - {359: 5405, 732: 5404}, + {361: 5406, 730: 5405}, + {513: 5408}, {513: 5407}, - {513: 5406}, {2119, 2119}, {2120, 2120}, // 2950 - {2116, 2116, 461: 2116, 2116, 2116, 467: 2116, 479: 2116, 5411, 491: 2116, 495: 2116, 559: 2116, 636: 2116, 1080: 5410}, - {384, 384, 4: 4132, 4134, 389, 4847, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 461: 384, 384, 384, 467: 384, 4131, 4168, 473: 2118, 479: 384, 384, 491: 384, 4845, 494: 2118, 384, 559: 384, 636: 384, 2118, 639: 4137, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4846}, - {2064, 2064, 461: 2064, 2064, 2064, 467: 2064, 479: 5000, 491: 2064, 495: 5001, 559: 2064, 636: 2064, 1021: 5412}, - {649: 4895}, - {2061, 2061, 461: 2061, 2061, 2061, 467: 5414, 491: 2061, 559: 2061, 636: 2061, 1154: 5413}, + {2116, 2116, 461: 2116, 2116, 2116, 467: 2116, 479: 2116, 5412, 491: 2116, 496: 2116, 559: 2116, 634: 2116, 1079: 5411}, + {384, 384, 4: 4133, 4135, 389, 4848, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 461: 384, 384, 384, 467: 384, 4132, 4169, 473: 2118, 479: 384, 384, 491: 384, 4846, 495: 2118, 384, 559: 384, 634: 384, 2118, 637: 4138, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4847}, + {2064, 2064, 461: 2064, 2064, 2064, 467: 2064, 479: 5001, 491: 2064, 496: 5002, 559: 2064, 634: 2064, 1019: 5413}, + {647: 4896}, + {2061, 2061, 461: 2061, 2061, 2061, 467: 5415, 491: 2061, 559: 2061, 634: 2061, 1153: 5414}, // 2955 - {2059, 2059, 461: 2059, 2507, 2506, 491: 2505, 559: 2504, 636: 2500, 699: 5419, 741: 5417, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 5418, 5416, 3777, 1175: 5415}, - {2060, 2060, 461: 2060, 2060, 2060, 491: 2060, 559: 2060, 636: 2060}, - {2121, 2121, 461: 5402, 1067: 5420}, + {2059, 2059, 461: 2059, 2508, 2507, 491: 2506, 559: 2505, 634: 2501, 697: 5420, 739: 5418, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 5419, 5417, 3778, 1174: 5416}, + {2060, 2060, 461: 2060, 2060, 2060, 491: 2060, 559: 2060, 634: 2060}, + {2121, 2121, 461: 5403, 1066: 5421}, {2058, 2058, 461: 2058}, {2057, 2057, 461: 2057, 469: 787, 477: 787, 787}, // 2960 {2056, 2056, 461: 2056}, - {2055, 2055, 461: 2055, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {2055, 2055, 461: 2055, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, {2123, 2123}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5354, 2674, 2675, 2673, 939: 5424}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5354, 2674, 2675, 2673, 939: 5423}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5355, 2675, 2676, 2674, 937: 5425}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5355, 2675, 2676, 2674, 937: 5424}, // 2965 - {685, 685, 7: 5356}, - {686, 686, 7: 5356}, + {685, 685, 7: 5357}, + {686, 686, 7: 5357}, {688, 688}, {687, 687}, {679, 679}, // 2970 {678, 678}, {677, 677}, - {253: 5431}, - {496: 2648, 725: 3948, 751: 5433, 1063: 5432}, - {691, 691, 7: 5434}, + {253: 5432}, + {497: 2649, 721: 3949, 749: 5434, 1062: 5433}, + {691, 691, 7: 5435}, // 2975 {666, 666, 7: 666}, - {496: 2648, 725: 3948, 751: 5435}, + {497: 2649, 721: 3949, 749: 5436}, {665, 665, 7: 665}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 5437}, - {692, 692, 7: 3821}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 5438}, + {692, 692, 7: 3822}, // 2980 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5442}, - {485: 5440}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 5441}, - {684, 684, 7: 3821}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5443, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5443}, + {485: 5441}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 5442}, + {684, 684, 7: 3822}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5444, 2675, 2676, 2674}, // 2985 {694, 694}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5445}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5446, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5446}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5447, 2675, 2676, 2674}, {695, 695}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 5460}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 5461}, // 2990 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5449}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5450, 2674, 2675, 2673}, - {696, 696, 462: 5453, 1039: 5452, 1200: 5451}, - {693, 693, 7: 5458}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5450}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5451, 2675, 2676, 2674}, + {696, 696, 462: 5454, 1037: 5453, 1199: 5452}, + {693, 693, 7: 5459}, {669, 669, 7: 669}, // 2995 - {496: 2648, 725: 3948, 751: 5454}, - {7: 5455}, - {496: 2648, 725: 3948, 751: 5456}, - {50: 5457}, + {497: 2649, 721: 3949, 749: 5455}, + {7: 5456}, + {497: 2649, 721: 3949, 749: 5457}, + {50: 5458}, {667, 667, 7: 667}, // 3000 - {462: 5453, 1039: 5459}, + {462: 5454, 1037: 5460}, {668, 668, 7: 668}, - {697, 697, 7: 3821}, - {157: 1601, 363: 5474, 387: 5475, 642: 1601, 1145: 5473}, - {701, 701, 157: 1443, 252: 5467, 5466, 642: 1443}, + {697, 697, 7: 3822}, + {157: 1601, 365: 5475, 387: 5476, 640: 1601, 1144: 5474}, + {701, 701, 157: 1443, 252: 5468, 5467, 640: 1443}, // 3005 - {676, 676, 157: 1424, 642: 1424}, - {157: 5465}, + {676, 676, 157: 1424, 640: 1424}, + {157: 5466}, {698, 698}, - {237, 237, 489: 2634, 496: 2648, 725: 3948, 751: 5471, 778: 2635, 5470}, - {362: 5468}, + {237, 237, 489: 2635, 497: 2649, 721: 3949, 749: 5472, 776: 2636, 5471}, + {364: 5469}, // 3010 - {496: 2648, 725: 3948, 751: 5433, 1063: 5469}, - {690, 690, 7: 5434}, + {497: 2649, 721: 3949, 749: 5434, 1062: 5470}, + {690, 690, 7: 5435}, {700, 700}, - {237, 237, 489: 2634, 778: 2635, 5472}, + {237, 237, 489: 2635, 776: 2636, 5473}, {699, 699}, // 3015 {689, 689}, - {496: 2648, 725: 5481}, - {334: 5477, 496: 2648, 641: 5478, 725: 5476}, + {497: 2649, 721: 5482}, + {336: 5478, 497: 2649, 639: 5479, 721: 5477}, {672, 672}, - {496: 2648, 725: 5480}, + {497: 2649, 721: 5481}, // 3020 - {496: 2648, 725: 5479}, + {497: 2649, 721: 5480}, {670, 670}, {671, 671}, {673, 673}, - {2: 259, 259, 259, 259, 259, 8: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 51: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 464: 259, 468: 259, 487: 1762, 532: 259, 642: 1762, 650: 1762}, + {2: 259, 259, 259, 259, 259, 8: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 51: 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 259, 464: 259, 468: 259, 487: 1762, 534: 259, 640: 1762, 648: 1762}, // 3025 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5586, 487: 1760, 642: 1760, 650: 1760, 653: 5585, 2674, 2675, 2673}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 5583, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 487: 1723, 642: 1723, 650: 1723, 653: 5493, 2674, 2675, 2673, 818: 5536}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 487: 1717, 642: 1717, 650: 1717, 653: 5493, 2674, 2675, 2673, 818: 5580}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 468: 5576, 487: 1715, 532: 3658, 642: 1715, 650: 1715, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 5575}, - {482: 5565, 487: 5564, 642: 1710, 650: 1710}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5587, 487: 1760, 640: 1760, 648: 1760, 651: 5586, 2675, 2676, 2674}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 5584, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 487: 1723, 640: 1723, 648: 1723, 651: 5494, 2675, 2676, 2674, 816: 5537}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 487: 1717, 640: 1717, 648: 1717, 651: 5494, 2675, 2676, 2674, 816: 5581}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 468: 5577, 487: 1715, 534: 3659, 640: 1715, 648: 1715, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 5576}, + {482: 5566, 487: 5565, 640: 1710, 648: 1710}, // 3030 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 5517, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 468: 5561, 487: 1701, 641: 5559, 1701, 650: 1701, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 5520, 1105: 5560, 1268: 5558}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 5556, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 487: 1699, 642: 1699, 650: 1699, 653: 5493, 2674, 2675, 2673, 818: 5533}, - {174: 5541, 487: 1682, 642: 1682, 650: 1682, 661: 5542, 908: 5540, 955: 5539}, - {779, 779, 7: 5529}, - {167: 5515}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 5518, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 468: 5562, 487: 1701, 639: 5560, 1701, 648: 1701, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 5521, 1104: 5561, 1266: 5559}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 5557, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 487: 1699, 640: 1699, 648: 1699, 651: 5494, 2675, 2676, 2674, 816: 5534}, + {174: 5542, 487: 1682, 640: 1682, 648: 1682, 659: 5543, 906: 5541, 953: 5540}, + {779, 779, 7: 5530}, + {167: 5516}, // 3035 - {487: 748, 642: 5513, 650: 748}, - {487: 5502, 650: 5503, 812: 5511}, - {487: 5502, 650: 5503, 812: 5506}, - {487: 5502, 650: 5503, 812: 5504}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 468: 5501, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 5500, 1162: 5499}, + {487: 748, 640: 5514, 648: 748}, + {487: 5503, 648: 5504, 810: 5512}, + {487: 5503, 648: 5504, 810: 5507}, + {487: 5503, 648: 5504, 810: 5505}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 468: 5502, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 5501, 1161: 5500}, // 3040 {726, 726, 7: 726}, {733, 733, 7: 733}, {732, 732, 7: 732}, {731, 731, 7: 731}, - {2: 750, 750, 750, 750, 750, 8: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 51: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 464: 750, 750, 750, 468: 750, 471: 750, 750, 474: 750, 750, 750, 481: 750, 491: 750, 494: 750, 750, 750, 532: 750, 555: 750, 750, 750, 750, 560: 750, 750, 750, 750, 750, 750, 567: 750, 750, 750, 750, 572: 750, 750, 575: 750, 577: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 638: 750}, + {2: 750, 750, 750, 750, 750, 8: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 51: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 464: 750, 750, 750, 468: 750, 471: 750, 750, 474: 750, 750, 750, 481: 750, 491: 750, 495: 750, 750, 750, 534: 750, 555: 750, 750, 750, 750, 560: 750, 750, 750, 750, 750, 750, 567: 750, 750, 750, 750, 572: 750, 750, 575: 750, 577: 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 750, 636: 750}, // 3045 - {2: 749, 749, 749, 749, 749, 8: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 51: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 464: 749, 749, 749, 468: 749, 471: 749, 749, 474: 749, 749, 749, 481: 749, 491: 749, 494: 749, 749, 749, 532: 749, 555: 749, 749, 749, 749, 560: 749, 749, 749, 749, 749, 749, 567: 749, 749, 749, 749, 572: 749, 749, 575: 749, 577: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 638: 749}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5505}, - {738, 738, 7: 738, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5510}, - {752, 752, 2910, 2758, 2794, 2912, 2685, 752, 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3763, 3153, 3236, 3152, 3149}, + {2: 749, 749, 749, 749, 749, 8: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 51: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 464: 749, 749, 749, 468: 749, 471: 749, 749, 474: 749, 749, 749, 481: 749, 491: 749, 495: 749, 749, 749, 534: 749, 555: 749, 749, 749, 749, 560: 749, 749, 749, 749, 749, 749, 567: 749, 749, 749, 749, 572: 749, 749, 575: 749, 577: 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 749, 636: 749}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5506}, + {738, 738, 7: 738, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5511}, + {752, 752, 2911, 2759, 2795, 2913, 2686, 752, 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3764, 3154, 3235, 3153, 3150}, // 3050 {753, 753, 7: 753}, {751, 751, 7: 751}, {739, 739, 7: 739}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5512}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5513}, {743, 743, 7: 743}, // 3055 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5514, 2674, 2675, 2673}, - {487: 747, 650: 747}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 5517, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 641: 5519, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 5520, 1105: 5518}, - {710, 710, 7: 710, 557: 1789, 640: 710, 657: 1789}, - {769, 769, 557: 1623, 640: 769, 657: 1623}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5515, 2675, 2676, 2674}, + {487: 747, 648: 747}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 5518, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 639: 5520, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 5521, 1104: 5519}, + {710, 710, 7: 710, 557: 1789, 638: 710, 655: 1789}, + {769, 769, 557: 1623, 638: 769, 655: 1623}, // 3060 - {640: 5527}, - {640: 768}, - {767, 767, 7: 5525, 640: 767}, - {711, 711, 7: 711, 557: 248, 640: 711, 657: 248}, - {705, 705, 7: 705, 640: 705}, + {638: 5528}, + {638: 768}, + {767, 767, 7: 5526, 638: 767}, + {711, 711, 7: 711, 557: 248, 638: 711, 655: 248}, + {705, 705, 7: 705, 638: 705}, // 3065 - {704, 704, 7: 704, 640: 704}, - {703, 703, 7: 703, 640: 703}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5526, 5522}, - {702, 702, 7: 702, 640: 702}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 5528}, + {704, 704, 7: 704, 638: 704}, + {703, 703, 7: 703, 638: 703}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5527, 5523}, + {702, 702, 7: 702, 638: 702}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 5529}, // 3070 - {770, 770, 7: 5094}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 5482, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 5485, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 5530, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 5531, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 5486, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 494: 3734, 557: 5496, 577: 5495, 637: 3732, 653: 5493, 2674, 2675, 2673, 761: 5497, 818: 5494, 962: 5532}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 487: 1723, 642: 1723, 650: 1723, 653: 5493, 2674, 2675, 2673, 818: 5536}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 487: 1699, 642: 1699, 650: 1699, 653: 5493, 2674, 2675, 2673, 818: 5533}, + {770, 770, 7: 5095}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 5483, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 5486, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 5531, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 5532, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 5487, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 495: 3735, 557: 5497, 577: 5496, 635: 3733, 651: 5494, 2675, 2676, 2674, 759: 5498, 816: 5495, 960: 5533}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 487: 1723, 640: 1723, 648: 1723, 651: 5494, 2675, 2676, 2674, 816: 5537}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 487: 1699, 640: 1699, 648: 1699, 651: 5494, 2675, 2676, 2674, 816: 5534}, {725, 725, 7: 725}, // 3075 - {487: 5502, 650: 5503, 812: 5534}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5535}, + {487: 5503, 648: 5504, 810: 5535}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5536}, {741, 741, 7: 741}, - {487: 5502, 650: 5503, 812: 5537}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5538}, + {487: 5503, 648: 5504, 810: 5538}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5539}, // 3080 {742, 742, 7: 742}, - {774, 774, 7: 5554}, + {774, 774, 7: 5555}, {763, 763, 7: 763}, - {340: 5546}, - {150: 5544, 677: 5543}, + {342: 5547}, + {150: 5545, 675: 5544}, // 3085 {760, 760, 7: 760}, - {759, 759, 7: 759, 663: 5265, 911: 5545}, + {759, 759, 7: 759, 661: 5266, 909: 5546}, {758, 758, 7: 758}, - {210: 5548, 370: 5550, 661: 5549, 1214: 5547}, + {210: 5549, 371: 5551, 659: 5550, 1213: 5548}, {761, 761, 7: 761}, // 3090 - {661: 5553}, - {310: 5551, 391: 5552}, + {659: 5554}, + {312: 5552, 391: 5553}, {754, 754, 7: 754}, {756, 756, 7: 756}, {755, 755, 7: 755}, // 3095 {757, 757, 7: 757}, - {174: 5541, 661: 5542, 908: 5555}, + {174: 5542, 659: 5543, 906: 5556}, {762, 762, 7: 762}, - {174: 5541, 487: 1682, 642: 1682, 650: 1682, 661: 5542, 908: 5540, 955: 5557}, - {775, 775, 7: 5554}, + {174: 5542, 487: 1682, 640: 1682, 648: 1682, 659: 5543, 906: 5541, 953: 5558}, + {775, 775, 7: 5555}, // 3100 {771, 771}, - {768, 768, 477: 5562}, + {768, 768, 477: 5563}, {765, 765}, {764, 764}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 5563}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 5564}, // 3105 - {766, 766, 7: 5525}, - {13: 5570, 464: 5569, 1081: 5574}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5566}, - {487: 5567}, - {13: 5570, 464: 5569, 1081: 5568}, + {766, 766, 7: 5526}, + {13: 5571, 464: 5570, 1080: 5575}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5567}, + {487: 5568}, + {13: 5571, 464: 5570, 1080: 5569}, // 3110 {777, 777}, {714, 714}, - {462: 5571}, - {464: 5124, 879: 5572}, - {50: 5573}, + {462: 5572}, + {464: 5125, 877: 5573}, + {50: 5574}, // 3115 {713, 713}, {778, 778}, - {737, 737, 7: 737, 473: 5577}, + {737, 737, 7: 737, 473: 5578}, {734, 734, 7: 734}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 468: 5578, 653: 3390, 2674, 2675, 2673, 728: 5579}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 468: 5579, 651: 3391, 2675, 2676, 2674, 726: 5580}, // 3120 {736, 736, 7: 736}, {735, 735, 7: 735}, - {487: 5502, 650: 5503, 812: 5581}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5582}, + {487: 5503, 648: 5504, 810: 5582}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5583}, {740, 740, 7: 740}, // 3125 - {174: 5541, 487: 1682, 642: 1682, 650: 1682, 661: 5542, 908: 5540, 955: 5584}, - {776, 776, 7: 5554}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5588, 2674, 2675, 2673, 889: 5595}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5588, 2674, 2675, 2673, 889: 5587}, - {487: 5502, 650: 5503, 812: 5593}, + {174: 5542, 487: 1682, 640: 1682, 648: 1682, 659: 5543, 906: 5541, 953: 5585}, + {776, 776, 7: 5555}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5589, 2675, 2676, 2674, 887: 5596}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5589, 2675, 2676, 2674, 887: 5588}, + {487: 5503, 648: 5504, 810: 5594}, // 3130 - {474: 5590, 487: 746, 642: 5589, 650: 746}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5588, 2674, 2675, 2673, 889: 5592}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5588, 2674, 2675, 2673, 889: 5591}, - {487: 744, 650: 744}, - {487: 745, 650: 745}, + {474: 5591, 487: 746, 640: 5590, 648: 746}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5589, 2675, 2676, 2674, 887: 5593}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5589, 2675, 2676, 2674, 887: 5592}, + {487: 744, 648: 744}, + {487: 745, 648: 745}, // 3135 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5594}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5595}, {772, 772}, - {487: 5502, 650: 5503, 812: 5596}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 5508, 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 5507, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5509, 828: 5597}, + {487: 5503, 648: 5504, 810: 5597}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 5509, 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 5508, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5510, 826: 5598}, {773, 773}, // 3140 - {640: 5607}, - {640: 5600}, - {259: 5601}, - {487: 5602}, - {464: 5603}, + {638: 5608}, + {638: 5601}, + {259: 5602}, + {487: 5603}, + {464: 5604}, // 3145 - {482: 5604}, - {258: 5605}, - {464: 5606}, + {482: 5605}, + {258: 5606}, + {464: 5607}, {780, 780}, - {259: 5608}, + {259: 5609}, // 3150 - {487: 5609}, - {464: 5610}, - {482: 5611}, - {258: 5612}, - {464: 5613}, + {487: 5610}, + {464: 5611}, + {482: 5612}, + {258: 5613}, + {464: 5614}, // 3155 {781, 781}, - {462: 2507, 491: 2505, 559: 2504, 636: 2500, 699: 5625, 741: 5624, 2501, 2502, 2503, 5626}, - {462: 1208, 491: 1208, 559: 1208, 636: 1208, 641: 3445, 735: 3443, 3444, 772: 5618, 775: 5619, 920: 5621, 950: 5623}, - {462: 1208, 491: 1208, 559: 1208, 636: 1208, 641: 3445, 735: 3443, 3444, 772: 5618, 775: 5619, 920: 5621, 950: 5622}, - {462: 1208, 491: 1208, 559: 1208, 636: 1208, 641: 3445, 735: 3443, 3444, 772: 5618, 775: 5619, 920: 5621, 950: 5620}, + {462: 2508, 491: 2506, 559: 2505, 634: 2501, 697: 5626, 739: 5625, 2502, 2503, 2504, 5627}, + {462: 1208, 491: 1208, 559: 1208, 634: 1208, 639: 3446, 733: 3444, 3445, 770: 5619, 773: 5620, 918: 5622, 948: 5624}, + {462: 1208, 491: 1208, 559: 1208, 634: 1208, 639: 3446, 733: 3444, 3445, 770: 5619, 773: 5620, 918: 5622, 948: 5623}, + {462: 1208, 491: 1208, 559: 1208, 634: 1208, 639: 3446, 733: 3444, 3445, 770: 5619, 773: 5620, 918: 5622, 948: 5621}, // 3160 - {2: 1211, 1211, 1211, 1211, 1211, 8: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 51: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 462: 1211, 464: 1211, 1211, 1211, 468: 1211, 471: 1211, 1211, 474: 1211, 1211, 1211, 481: 1211, 491: 1211, 494: 1211, 1211, 1211, 503: 1211, 512: 1211, 532: 1211, 555: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 567: 1211, 1211, 1211, 1211, 572: 1211, 1211, 575: 1211, 577: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 636: 1211, 638: 1211, 641: 1211, 735: 1211, 1211, 738: 1211, 1211, 1211, 749: 1211, 758: 1211, 1211, 1211}, - {462: 1207, 491: 1207, 559: 1207, 636: 1207}, - {462: 783, 491: 783, 559: 783, 636: 783}, - {462: 782, 491: 782, 559: 782, 636: 782}, - {462: 784, 491: 784, 559: 784, 636: 784}, + {2: 1211, 1211, 1211, 1211, 1211, 8: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 51: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 462: 1211, 464: 1211, 1211, 1211, 468: 1211, 471: 1211, 1211, 474: 1211, 1211, 1211, 481: 1211, 491: 1211, 495: 1211, 1211, 1211, 503: 1211, 512: 1211, 534: 1211, 555: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 567: 1211, 1211, 1211, 1211, 572: 1211, 1211, 575: 1211, 577: 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 1211, 634: 1211, 636: 1211, 639: 1211, 733: 1211, 1211, 736: 1211, 1211, 1211, 747: 1211, 756: 1211, 1211, 1211}, + {462: 1207, 491: 1207, 559: 1207, 634: 1207}, + {462: 783, 491: 783, 559: 783, 634: 783}, + {462: 782, 491: 782, 559: 782, 634: 782}, + {462: 784, 491: 784, 559: 784, 634: 784}, // 3165 - {462: 785, 491: 785, 559: 785, 636: 785}, + {462: 785, 491: 785, 559: 785, 634: 785}, {797, 797, 50: 797, 461: 797, 463: 797, 469: 787, 797, 477: 787, 787}, - {796, 796, 50: 796, 461: 796, 463: 796, 469: 786, 796, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 5627, 5628}, + {796, 796, 50: 796, 461: 796, 463: 796, 469: 786, 796, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 5628, 5629}, {469: 788, 477: 788, 788}, - {795, 795, 50: 795, 461: 795, 463: 795, 470: 795, 483: 2640, 488: 2641, 757: 5629}, + {795, 795, 50: 795, 461: 795, 463: 795, 470: 795, 483: 2641, 488: 2642, 755: 5630}, // 3170 {794, 794, 50: 794, 461: 794, 463: 794, 470: 794}, {793, 793, 50: 793, 461: 793, 463: 793, 470: 793}, - {50: 3871, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, - {7: 5645, 462: 969, 491: 969, 559: 969, 636: 969, 644: 969, 732: 969}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5634, 2674, 2675, 2673, 917: 5633, 1143: 5644}, + {50: 3872, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, + {7: 5646, 462: 969, 491: 969, 559: 969, 634: 969, 642: 969, 730: 969}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5635, 2675, 2676, 2674, 915: 5634, 1142: 5645}, // 3175 - {7: 966, 462: 966, 491: 966, 559: 966, 636: 966, 644: 966, 732: 966}, - {462: 5635, 467: 2260, 1202: 5636}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5640, 2674, 2675, 2673, 863: 5639}, - {467: 5637}, - {462: 2507, 699: 5638}, + {7: 966, 462: 966, 491: 966, 559: 966, 634: 966, 642: 966, 730: 966}, + {462: 5636, 467: 2261, 1201: 5637}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5641, 2675, 2676, 2674, 861: 5640}, + {467: 5638}, + {462: 2508, 697: 5639}, // 3180 - {7: 965, 462: 965, 491: 965, 559: 965, 636: 965, 644: 965, 732: 965}, - {7: 5642, 50: 5641}, - {2258, 2258, 7: 2258, 50: 2258, 463: 2258}, - {467: 2259}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5643, 2674, 2675, 2673}, + {7: 965, 462: 965, 491: 965, 559: 965, 634: 965, 642: 965, 730: 965}, + {7: 5643, 50: 5642}, + {2259, 2259, 7: 2259, 50: 2259, 463: 2259}, + {467: 2260}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5644, 2675, 2676, 2674}, // 3185 - {2257, 2257, 7: 2257, 50: 2257, 463: 2257}, - {7: 5645, 462: 968, 491: 968, 559: 968, 636: 968, 644: 968, 732: 968}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5634, 2674, 2675, 2673, 917: 5646}, - {7: 967, 462: 967, 491: 967, 559: 967, 636: 967, 644: 967, 732: 967}, - {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2637, 756: 2638, 799: 5648}, + {2258, 2258, 7: 2258, 50: 2258, 463: 2258}, + {7: 5646, 462: 968, 491: 968, 559: 968, 634: 968, 642: 968, 730: 968}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5635, 2675, 2676, 2674, 915: 5647}, + {7: 967, 462: 967, 491: 967, 559: 967, 634: 967, 642: 967, 730: 967}, + {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2638, 754: 2639, 797: 5649}, // 3190 - {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2640, 841, 841, 488: 2641, 757: 2642, 816: 5649}, - {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3809, 484: 812, 3810, 874: 5650}, - {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3835, 875: 5651}, + {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2641, 841, 841, 488: 2642, 755: 2643, 814: 5650}, + {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3810, 484: 812, 3811, 872: 5651}, + {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3836, 873: 5652}, {973, 973, 50: 973, 461: 973, 463: 973, 469: 973, 973, 477: 973, 973}, - {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2640, 841, 841, 488: 2641, 757: 2642, 816: 5653}, + {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2641, 841, 841, 488: 2642, 755: 2643, 814: 5654}, // 3195 - {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3809, 484: 812, 3810, 874: 5654}, - {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3835, 875: 5655}, + {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3810, 484: 812, 3811, 872: 5655}, + {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3836, 873: 5656}, {974, 974, 50: 974, 461: 974, 463: 974, 469: 974, 974, 477: 974, 974}, - {649: 5663}, - {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2637, 756: 2638, 799: 5659}, + {647: 5664}, + {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2638, 754: 2639, 797: 5660}, // 3200 {819, 819, 50: 819, 461: 819, 463: 819, 469: 819, 819, 477: 819, 819, 482: 819, 819, 819, 819, 488: 819, 490: 819, 504: 819, 819}, - {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2640, 841, 841, 488: 2641, 757: 2642, 816: 5660}, - {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3809, 484: 812, 3810, 874: 5661}, - {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3835, 875: 5662}, + {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2641, 841, 841, 488: 2642, 755: 2643, 814: 5661}, + {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3810, 484: 812, 3811, 872: 5662}, + {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3836, 873: 5663}, {975, 975, 50: 975, 461: 975, 463: 975, 469: 975, 975, 477: 975, 975}, // 3205 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 2666, 857: 3142, 887: 5664}, - {1823, 1823, 7: 3311, 50: 1823, 461: 1823, 463: 1823, 469: 1823, 1823, 477: 1823, 1823, 482: 1823, 1823, 1823, 1823, 488: 1823, 490: 1823, 504: 1823, 1823}, - {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 488: 237, 2634, 237, 502: 237, 778: 2635, 5690}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 568: 5204, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5211, 838: 5201, 876: 5675, 1181: 5674, 1295: 5673}, - {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5656, 926: 5658, 949: 5668}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 2667, 855: 3143, 885: 5665}, + {1823, 1823, 7: 3312, 50: 1823, 461: 1823, 463: 1823, 469: 1823, 1823, 477: 1823, 1823, 482: 1823, 1823, 1823, 1823, 488: 1823, 490: 1823, 504: 1823, 1823}, + {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 488: 237, 2635, 237, 502: 237, 776: 2636, 5691}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 568: 5205, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5212, 836: 5202, 874: 5676, 1180: 5675, 1293: 5674}, + {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5657, 924: 5659, 947: 5669}, // 3210 - {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2637, 756: 2638, 799: 5669}, - {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2640, 841, 841, 488: 2641, 757: 2642, 816: 5670}, - {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3809, 484: 812, 3810, 874: 5671}, - {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3835, 875: 5672}, + {1265, 1265, 50: 1265, 461: 1265, 463: 1265, 469: 1265, 1265, 477: 1265, 1265, 482: 1265, 1265, 1265, 1265, 488: 1265, 490: 2638, 754: 2639, 797: 5670}, + {841, 841, 50: 841, 461: 841, 463: 841, 469: 841, 841, 477: 841, 841, 482: 841, 2641, 841, 841, 488: 2642, 755: 2643, 814: 5671}, + {812, 812, 50: 812, 461: 812, 463: 812, 469: 812, 812, 477: 812, 812, 482: 3810, 484: 812, 3811, 872: 5672}, + {818, 818, 50: 818, 461: 818, 463: 818, 469: 818, 818, 477: 818, 818, 484: 3836, 873: 5673}, {976, 976, 50: 976, 461: 976, 463: 976, 469: 976, 976, 477: 976, 976}, // 3215 - {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 488: 237, 2634, 237, 502: 237, 504: 237, 237, 778: 2635, 5676}, + {237, 237, 50: 237, 461: 237, 463: 237, 469: 237, 237, 477: 237, 237, 482: 237, 237, 237, 237, 488: 237, 2635, 237, 502: 237, 504: 237, 237, 776: 2636, 5677}, {964, 964, 50: 964, 461: 964, 463: 964, 469: 964, 964, 477: 964, 964, 482: 964, 964, 964, 964, 488: 964, 964, 964, 502: 964}, - {904, 904, 7: 5255, 50: 904, 461: 904, 463: 904, 469: 904, 904, 477: 904, 904, 482: 904, 904, 904, 904, 488: 904, 904, 904, 502: 904, 504: 904, 904}, - {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5656, 504: 820, 820, 926: 5658, 949: 5677}, - {1822, 1822, 50: 1822, 461: 1822, 463: 1822, 469: 1822, 1822, 477: 1822, 1822, 482: 1822, 1822, 1822, 1822, 488: 1822, 490: 1822, 504: 1822, 5678, 1201: 5679}, + {904, 904, 7: 5256, 50: 904, 461: 904, 463: 904, 469: 904, 904, 477: 904, 904, 482: 904, 904, 904, 904, 488: 904, 904, 904, 502: 904, 504: 904, 904}, + {820, 820, 50: 820, 461: 820, 463: 820, 469: 820, 820, 477: 820, 820, 482: 820, 820, 820, 820, 488: 820, 490: 820, 502: 5657, 504: 820, 820, 924: 5659, 947: 5678}, + {1822, 1822, 50: 1822, 461: 1822, 463: 1822, 469: 1822, 1822, 477: 1822, 1822, 482: 1822, 1822, 1822, 1822, 488: 1822, 490: 1822, 504: 1822, 5679, 1200: 5680}, // 3220 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5689}, - {963, 963, 50: 963, 461: 963, 463: 963, 469: 963, 963, 477: 963, 963, 482: 963, 963, 963, 963, 488: 963, 490: 963, 504: 5681, 1318: 5680}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5690}, + {963, 963, 50: 963, 461: 963, 463: 963, 469: 963, 963, 477: 963, 963, 482: 963, 963, 963, 963, 488: 963, 490: 963, 504: 5682, 1316: 5681}, {989, 989, 50: 989, 461: 989, 463: 989, 469: 989, 989, 477: 989, 989, 482: 989, 989, 989, 989, 488: 989, 490: 989}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3299, 2674, 2675, 2673, 910: 5684, 1139: 5683, 1319: 5682}, - {962, 962, 7: 5687, 50: 962, 461: 962, 463: 962, 469: 962, 962, 477: 962, 962, 482: 962, 962, 962, 962, 488: 962, 490: 962}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3300, 2675, 2676, 2674, 908: 5685, 1138: 5684, 1317: 5683}, + {962, 962, 7: 5688, 50: 962, 461: 962, 463: 962, 469: 962, 962, 477: 962, 962, 482: 962, 962, 962, 962, 488: 962, 490: 962}, // 3225 {961, 961, 7: 961, 50: 961, 461: 961, 463: 961, 469: 961, 961, 477: 961, 961, 482: 961, 961, 961, 961, 488: 961, 490: 961}, - {467: 5685}, - {462: 3300, 1141: 5686}, + {467: 5686}, + {462: 3301, 1140: 5687}, {959, 959, 7: 959, 50: 959, 461: 959, 463: 959, 469: 959, 959, 477: 959, 959, 482: 959, 959, 959, 959, 488: 959, 490: 959}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3299, 2674, 2675, 2673, 910: 5684, 1139: 5688}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3300, 2675, 2676, 2674, 908: 5685, 1138: 5689}, // 3230 {960, 960, 7: 960, 50: 960, 461: 960, 463: 960, 469: 960, 960, 477: 960, 960, 482: 960, 960, 960, 960, 488: 960, 490: 960}, - {1821, 1821, 50: 1821, 461: 1821, 463: 1821, 469: 1821, 1821, 477: 1821, 1821, 482: 1821, 1821, 1821, 1821, 488: 1821, 490: 1821, 493: 3247, 497: 3245, 3246, 3244, 3242, 504: 1821, 723: 3243, 3241}, + {1821, 1821, 50: 1821, 461: 1821, 463: 1821, 469: 1821, 1821, 477: 1821, 1821, 482: 1821, 1821, 1821, 1821, 488: 1821, 490: 1821, 494: 3246, 498: 3244, 3245, 3243, 3241, 504: 1821, 722: 3242, 3240}, {990, 990, 50: 990, 461: 990, 463: 990, 469: 990, 990, 477: 990, 990, 482: 990, 990, 990, 990, 488: 990, 490: 990, 502: 990}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 512: 5707, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 5708, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5706, 1029: 5709, 1190: 5710, 1263: 5711}, - {2: 839, 839, 839, 839, 839, 8: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 51: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 462: 839, 464: 839, 839, 839, 468: 839, 471: 839, 839, 474: 839, 839, 839, 481: 839, 491: 839, 494: 839, 839, 839, 503: 839, 512: 839, 532: 839, 555: 839, 839, 839, 839, 560: 839, 839, 839, 839, 839, 839, 567: 839, 839, 839, 839, 572: 839, 839, 575: 839, 577: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 638: 839, 641: 839, 735: 839, 839, 738: 839, 839, 839, 749: 839, 758: 839, 839, 839}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 512: 5708, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 5709, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5707, 1027: 5710, 1189: 5711, 1261: 5712}, + {2: 839, 839, 839, 839, 839, 8: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 51: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 462: 839, 464: 839, 839, 839, 468: 839, 471: 839, 839, 474: 839, 839, 839, 481: 839, 491: 839, 495: 839, 839, 839, 503: 839, 512: 839, 534: 839, 555: 839, 839, 839, 839, 560: 839, 839, 839, 839, 839, 839, 567: 839, 839, 839, 839, 572: 839, 839, 575: 839, 577: 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 839, 636: 839, 639: 839, 733: 839, 839, 736: 839, 839, 839, 747: 839, 756: 839, 839, 839}, // 3235 - {2: 838, 838, 838, 838, 838, 8: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 51: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 462: 838, 464: 838, 838, 838, 468: 838, 471: 838, 838, 474: 838, 838, 838, 481: 838, 491: 838, 494: 838, 838, 838, 503: 838, 512: 838, 532: 838, 555: 838, 838, 838, 838, 560: 838, 838, 838, 838, 838, 838, 567: 838, 838, 838, 838, 572: 838, 838, 575: 838, 577: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 638: 838, 641: 838, 735: 838, 838, 738: 838, 838, 838, 749: 838, 758: 838, 838, 838}, - {2: 837, 837, 837, 837, 837, 8: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 51: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 462: 837, 464: 837, 837, 837, 468: 837, 471: 837, 837, 474: 837, 837, 837, 481: 837, 491: 837, 494: 837, 837, 837, 503: 837, 512: 837, 532: 837, 555: 837, 837, 837, 837, 560: 837, 837, 837, 837, 837, 837, 567: 837, 837, 837, 837, 572: 837, 837, 575: 837, 577: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 638: 837, 641: 837, 735: 837, 837, 738: 837, 837, 837, 749: 837, 758: 837, 837, 837}, - {2: 836, 836, 836, 836, 836, 8: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 51: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 462: 836, 464: 836, 836, 836, 468: 836, 471: 836, 836, 474: 836, 836, 836, 481: 836, 491: 836, 494: 836, 836, 836, 503: 836, 512: 836, 532: 836, 555: 836, 836, 836, 836, 560: 836, 836, 836, 836, 836, 836, 567: 836, 836, 836, 836, 572: 836, 836, 575: 836, 577: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 638: 836, 641: 836, 735: 836, 836, 738: 836, 836, 836, 749: 836, 758: 836, 836, 836}, - {2: 835, 835, 835, 835, 835, 8: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 51: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 462: 835, 464: 835, 835, 835, 468: 835, 471: 835, 835, 474: 835, 835, 835, 481: 835, 491: 835, 494: 835, 835, 835, 503: 835, 512: 835, 532: 835, 555: 835, 835, 835, 835, 560: 835, 835, 835, 835, 835, 835, 567: 835, 835, 835, 835, 572: 835, 835, 575: 835, 577: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 638: 835, 641: 835, 735: 835, 835, 738: 835, 835, 835, 749: 835, 758: 835, 835, 835}, - {2: 834, 834, 834, 834, 834, 8: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 51: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 462: 834, 464: 834, 834, 834, 468: 834, 471: 834, 834, 474: 834, 834, 834, 481: 834, 491: 834, 494: 834, 834, 834, 503: 834, 512: 834, 532: 834, 555: 834, 834, 834, 834, 560: 834, 834, 834, 834, 834, 834, 567: 834, 834, 834, 834, 572: 834, 834, 575: 834, 577: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 638: 834, 641: 834, 735: 834, 834, 738: 834, 834, 834, 749: 834, 758: 834, 834, 834}, + {2: 838, 838, 838, 838, 838, 8: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 51: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 462: 838, 464: 838, 838, 838, 468: 838, 471: 838, 838, 474: 838, 838, 838, 481: 838, 491: 838, 495: 838, 838, 838, 503: 838, 512: 838, 534: 838, 555: 838, 838, 838, 838, 560: 838, 838, 838, 838, 838, 838, 567: 838, 838, 838, 838, 572: 838, 838, 575: 838, 577: 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 838, 636: 838, 639: 838, 733: 838, 838, 736: 838, 838, 838, 747: 838, 756: 838, 838, 838}, + {2: 837, 837, 837, 837, 837, 8: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 51: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 462: 837, 464: 837, 837, 837, 468: 837, 471: 837, 837, 474: 837, 837, 837, 481: 837, 491: 837, 495: 837, 837, 837, 503: 837, 512: 837, 534: 837, 555: 837, 837, 837, 837, 560: 837, 837, 837, 837, 837, 837, 567: 837, 837, 837, 837, 572: 837, 837, 575: 837, 577: 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 837, 636: 837, 639: 837, 733: 837, 837, 736: 837, 837, 837, 747: 837, 756: 837, 837, 837}, + {2: 836, 836, 836, 836, 836, 8: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 51: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 462: 836, 464: 836, 836, 836, 468: 836, 471: 836, 836, 474: 836, 836, 836, 481: 836, 491: 836, 495: 836, 836, 836, 503: 836, 512: 836, 534: 836, 555: 836, 836, 836, 836, 560: 836, 836, 836, 836, 836, 836, 567: 836, 836, 836, 836, 572: 836, 836, 575: 836, 577: 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 836, 636: 836, 639: 836, 733: 836, 836, 736: 836, 836, 836, 747: 836, 756: 836, 836, 836}, + {2: 835, 835, 835, 835, 835, 8: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 51: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 462: 835, 464: 835, 835, 835, 468: 835, 471: 835, 835, 474: 835, 835, 835, 481: 835, 491: 835, 495: 835, 835, 835, 503: 835, 512: 835, 534: 835, 555: 835, 835, 835, 835, 560: 835, 835, 835, 835, 835, 835, 567: 835, 835, 835, 835, 572: 835, 835, 575: 835, 577: 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 835, 636: 835, 639: 835, 733: 835, 835, 736: 835, 835, 835, 747: 835, 756: 835, 835, 835}, + {2: 834, 834, 834, 834, 834, 8: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 51: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 462: 834, 464: 834, 834, 834, 468: 834, 471: 834, 834, 474: 834, 834, 834, 481: 834, 491: 834, 495: 834, 834, 834, 503: 834, 512: 834, 534: 834, 555: 834, 834, 834, 834, 560: 834, 834, 834, 834, 834, 834, 567: 834, 834, 834, 834, 572: 834, 834, 575: 834, 577: 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 834, 636: 834, 639: 834, 733: 834, 834, 736: 834, 834, 834, 747: 834, 756: 834, 834, 834}, // 3240 - {2: 833, 833, 833, 833, 833, 8: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 51: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 462: 833, 464: 833, 833, 833, 468: 833, 471: 833, 833, 474: 833, 833, 833, 481: 833, 491: 833, 494: 833, 833, 833, 503: 833, 512: 833, 532: 833, 555: 833, 833, 833, 833, 560: 833, 833, 833, 833, 833, 833, 567: 833, 833, 833, 833, 572: 833, 833, 575: 833, 577: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 638: 833, 641: 833, 735: 833, 833, 738: 833, 833, 833, 749: 833, 758: 833, 833, 833}, - {2: 832, 832, 832, 832, 832, 8: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 51: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 462: 832, 464: 832, 832, 832, 468: 832, 471: 832, 832, 474: 832, 832, 832, 481: 832, 491: 832, 494: 832, 832, 832, 503: 832, 512: 832, 532: 832, 555: 832, 832, 832, 832, 560: 832, 832, 832, 832, 832, 832, 567: 832, 832, 832, 832, 572: 832, 832, 575: 832, 577: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 638: 832, 641: 832, 735: 832, 832, 738: 832, 832, 832, 749: 832, 758: 832, 832, 832}, - {2: 831, 831, 831, 831, 831, 8: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 51: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 462: 831, 464: 831, 831, 831, 468: 831, 471: 831, 831, 474: 831, 831, 831, 481: 831, 491: 831, 494: 831, 831, 831, 503: 831, 512: 831, 532: 831, 555: 831, 831, 831, 831, 560: 831, 831, 831, 831, 831, 831, 567: 831, 831, 831, 831, 572: 831, 831, 575: 831, 577: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 638: 831, 641: 831, 735: 831, 831, 738: 831, 831, 831, 749: 831, 758: 831, 831, 831}, - {2: 829, 829, 829, 829, 829, 8: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 51: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 5697, 5703, 5704, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 462: 829, 464: 829, 829, 829, 468: 829, 471: 829, 829, 474: 829, 829, 829, 481: 829, 491: 829, 494: 829, 829, 829, 503: 5700, 512: 829, 532: 829, 555: 829, 829, 829, 829, 560: 829, 829, 829, 829, 829, 829, 567: 829, 829, 829, 829, 572: 829, 829, 575: 829, 577: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 638: 829, 641: 3445, 735: 3443, 3444, 738: 5197, 5196, 5195, 749: 5192, 758: 5696, 5699, 5695, 772: 5618, 775: 5693, 825: 5694, 853: 5692, 1103: 5705, 5698}, - {2: 827, 827, 827, 827, 827, 8: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 51: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 462: 827, 464: 827, 827, 827, 468: 827, 471: 827, 827, 474: 827, 827, 827, 481: 827, 491: 827, 494: 827, 827, 827, 503: 827, 512: 827, 532: 827, 555: 827, 827, 827, 827, 560: 827, 827, 827, 827, 827, 827, 567: 827, 827, 827, 827, 572: 827, 827, 575: 827, 577: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 638: 827, 641: 827, 735: 827, 827, 738: 827, 827, 827, 749: 827, 758: 827, 827, 827}, + {2: 833, 833, 833, 833, 833, 8: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 51: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 462: 833, 464: 833, 833, 833, 468: 833, 471: 833, 833, 474: 833, 833, 833, 481: 833, 491: 833, 495: 833, 833, 833, 503: 833, 512: 833, 534: 833, 555: 833, 833, 833, 833, 560: 833, 833, 833, 833, 833, 833, 567: 833, 833, 833, 833, 572: 833, 833, 575: 833, 577: 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 833, 636: 833, 639: 833, 733: 833, 833, 736: 833, 833, 833, 747: 833, 756: 833, 833, 833}, + {2: 832, 832, 832, 832, 832, 8: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 51: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 462: 832, 464: 832, 832, 832, 468: 832, 471: 832, 832, 474: 832, 832, 832, 481: 832, 491: 832, 495: 832, 832, 832, 503: 832, 512: 832, 534: 832, 555: 832, 832, 832, 832, 560: 832, 832, 832, 832, 832, 832, 567: 832, 832, 832, 832, 572: 832, 832, 575: 832, 577: 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 832, 636: 832, 639: 832, 733: 832, 832, 736: 832, 832, 832, 747: 832, 756: 832, 832, 832}, + {2: 831, 831, 831, 831, 831, 8: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 51: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 462: 831, 464: 831, 831, 831, 468: 831, 471: 831, 831, 474: 831, 831, 831, 481: 831, 491: 831, 495: 831, 831, 831, 503: 831, 512: 831, 534: 831, 555: 831, 831, 831, 831, 560: 831, 831, 831, 831, 831, 831, 567: 831, 831, 831, 831, 572: 831, 831, 575: 831, 577: 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 831, 636: 831, 639: 831, 733: 831, 831, 736: 831, 831, 831, 747: 831, 756: 831, 831, 831}, + {2: 829, 829, 829, 829, 829, 8: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 51: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 5698, 5704, 5705, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 462: 829, 464: 829, 829, 829, 468: 829, 471: 829, 829, 474: 829, 829, 829, 481: 829, 491: 829, 495: 829, 829, 829, 503: 5701, 512: 829, 534: 829, 555: 829, 829, 829, 829, 560: 829, 829, 829, 829, 829, 829, 567: 829, 829, 829, 829, 572: 829, 829, 575: 829, 577: 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 829, 636: 829, 639: 3446, 733: 3444, 3445, 736: 5198, 5197, 5196, 747: 5193, 756: 5697, 5700, 5696, 770: 5619, 773: 5694, 823: 5695, 851: 5693, 1102: 5706, 5699}, + {2: 827, 827, 827, 827, 827, 8: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 51: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 462: 827, 464: 827, 827, 827, 468: 827, 471: 827, 827, 474: 827, 827, 827, 481: 827, 491: 827, 495: 827, 827, 827, 503: 827, 512: 827, 534: 827, 555: 827, 827, 827, 827, 560: 827, 827, 827, 827, 827, 827, 567: 827, 827, 827, 827, 572: 827, 827, 575: 827, 577: 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 827, 636: 827, 639: 827, 733: 827, 827, 736: 827, 827, 827, 747: 827, 756: 827, 827, 827}, // 3245 - {2: 823, 823, 823, 823, 823, 8: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 51: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 462: 823, 464: 823, 823, 823, 468: 823, 471: 823, 823, 474: 823, 823, 823, 481: 823, 491: 823, 494: 823, 823, 823, 503: 823, 512: 823, 532: 823, 555: 823, 823, 823, 823, 560: 823, 823, 823, 823, 823, 823, 567: 823, 823, 823, 823, 572: 823, 823, 575: 823, 577: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 638: 823, 641: 823, 735: 823, 823, 738: 823, 823, 823, 749: 823, 758: 823, 823, 823}, - {2: 822, 822, 822, 822, 822, 8: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 51: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 462: 822, 464: 822, 822, 822, 468: 822, 471: 822, 822, 474: 822, 822, 822, 481: 822, 491: 822, 494: 822, 822, 822, 503: 822, 512: 822, 532: 822, 555: 822, 822, 822, 822, 560: 822, 822, 822, 822, 822, 822, 567: 822, 822, 822, 822, 572: 822, 822, 575: 822, 577: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 638: 822, 641: 822, 735: 822, 822, 738: 822, 822, 822, 749: 822, 758: 822, 822, 822}, - {2: 828, 828, 828, 828, 828, 8: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 51: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 462: 828, 464: 828, 828, 828, 468: 828, 471: 828, 828, 474: 828, 828, 828, 481: 828, 491: 828, 494: 828, 828, 828, 503: 828, 512: 828, 532: 828, 555: 828, 828, 828, 828, 560: 828, 828, 828, 828, 828, 828, 567: 828, 828, 828, 828, 572: 828, 828, 575: 828, 577: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 638: 828, 641: 828, 735: 828, 828, 738: 828, 828, 828, 749: 828, 758: 828, 828, 828}, - {1831, 1831, 2910, 2758, 2794, 2912, 2685, 1831, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1831, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 1831, 463: 1831, 5723, 467: 5722, 469: 1831, 1831, 477: 1831, 1831, 482: 1831, 1831, 1831, 1831, 1831, 488: 1831, 1831, 1831, 493: 3247, 497: 3245, 3246, 3244, 3242, 502: 1831, 653: 5721, 2674, 2675, 2673, 723: 3243, 3241, 1187: 5720, 5719}, + {2: 823, 823, 823, 823, 823, 8: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 51: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 462: 823, 464: 823, 823, 823, 468: 823, 471: 823, 823, 474: 823, 823, 823, 481: 823, 491: 823, 495: 823, 823, 823, 503: 823, 512: 823, 534: 823, 555: 823, 823, 823, 823, 560: 823, 823, 823, 823, 823, 823, 567: 823, 823, 823, 823, 572: 823, 823, 575: 823, 577: 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 823, 636: 823, 639: 823, 733: 823, 823, 736: 823, 823, 823, 747: 823, 756: 823, 823, 823}, + {2: 822, 822, 822, 822, 822, 8: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 51: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 462: 822, 464: 822, 822, 822, 468: 822, 471: 822, 822, 474: 822, 822, 822, 481: 822, 491: 822, 495: 822, 822, 822, 503: 822, 512: 822, 534: 822, 555: 822, 822, 822, 822, 560: 822, 822, 822, 822, 822, 822, 567: 822, 822, 822, 822, 572: 822, 822, 575: 822, 577: 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 822, 636: 822, 639: 822, 733: 822, 822, 736: 822, 822, 822, 747: 822, 756: 822, 822, 822}, + {2: 828, 828, 828, 828, 828, 8: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 51: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 462: 828, 464: 828, 828, 828, 468: 828, 471: 828, 828, 474: 828, 828, 828, 481: 828, 491: 828, 495: 828, 828, 828, 503: 828, 512: 828, 534: 828, 555: 828, 828, 828, 828, 560: 828, 828, 828, 828, 828, 828, 567: 828, 828, 828, 828, 572: 828, 828, 575: 828, 577: 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 828, 636: 828, 639: 828, 733: 828, 828, 736: 828, 828, 828, 747: 828, 756: 828, 828, 828}, + {1831, 1831, 2911, 2759, 2795, 2913, 2686, 1831, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1831, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 1831, 463: 1831, 5724, 467: 5723, 469: 1831, 1831, 477: 1831, 1831, 482: 1831, 1831, 1831, 1831, 1831, 488: 1831, 1831, 1831, 494: 3246, 498: 3244, 3245, 3243, 3241, 1831, 651: 5722, 2675, 2676, 2674, 722: 3242, 3240, 1186: 5721, 5720}, {1835, 1835, 7: 1835, 50: 1835, 461: 1835, 463: 1835, 469: 1835, 1835, 477: 1835, 1835, 482: 1835, 1835, 1835, 1835, 1835, 488: 1835, 1835, 1835, 502: 1835}, // 3250 - {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 463: 1248, 1248, 1248, 1248, 1248, 469: 1248, 1248, 473: 1248, 1248, 1248, 1248, 1248, 1248, 482: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 493: 1248, 497: 1248, 1248, 1248, 1248, 502: 1248, 510: 1248, 512: 1248, 533: 1248, 536: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 571: 1248, 642: 5714, 645: 1248, 1248}, + {1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 463: 1248, 1248, 1248, 1248, 1248, 469: 1248, 1248, 473: 1248, 1248, 1248, 1248, 1248, 1248, 482: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 494: 1248, 498: 1248, 1248, 1248, 1248, 1248, 510: 1248, 512: 1248, 532: 1248, 536: 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 1248, 571: 1248, 640: 5715, 643: 1248, 1248}, {1825, 1825, 7: 1825, 50: 1825, 461: 1825, 463: 1825, 469: 1825, 1825, 477: 1825, 1825, 482: 1825, 1825, 1825, 1825, 1825, 488: 1825, 1825, 1825, 502: 1825}, - {821, 821, 7: 5712, 50: 821, 461: 821, 463: 821, 469: 821, 821, 477: 821, 821, 482: 821, 821, 821, 821, 821, 488: 821, 821, 821, 502: 821}, + {821, 821, 7: 5713, 50: 821, 461: 821, 463: 821, 469: 821, 821, 477: 821, 821, 482: 821, 821, 821, 821, 821, 488: 821, 821, 821, 502: 821}, {991, 991, 50: 991, 461: 991, 463: 991, 469: 991, 991, 477: 991, 991, 482: 991, 991, 991, 991, 991, 488: 991, 991, 991, 502: 991}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 512: 5707, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 5708, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5706, 1029: 5713}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 512: 5708, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 5709, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5707, 1027: 5714}, // 3255 {1824, 1824, 7: 1824, 50: 1824, 461: 1824, 463: 1824, 469: 1824, 1824, 477: 1824, 1824, 482: 1824, 1824, 1824, 1824, 1824, 488: 1824, 1824, 1824, 502: 1824}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5715, 653: 5716, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5716, 651: 5717, 2675, 2676, 2674}, {1834, 1834, 7: 1834, 50: 1834, 461: 1834, 463: 1834, 469: 1834, 1834, 477: 1834, 1834, 482: 1834, 1834, 1834, 1834, 1834, 488: 1834, 1834, 1834, 502: 1834}, - {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3887, 1247, 1247, 1247, 1247, 1247, 469: 1247, 1247, 473: 1247, 1247, 1247, 1247, 1247, 1247, 482: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 493: 1247, 497: 1247, 1247, 1247, 1247, 502: 1247, 510: 1247, 512: 1247, 533: 1247, 536: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 571: 1247, 642: 5717, 645: 1247, 1247}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5718, 653: 3683, 2674, 2675, 2673}, + {1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 3888, 1247, 1247, 1247, 1247, 1247, 469: 1247, 1247, 473: 1247, 1247, 1247, 1247, 1247, 1247, 482: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 494: 1247, 498: 1247, 1247, 1247, 1247, 1247, 510: 1247, 512: 1247, 532: 1247, 536: 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 1247, 571: 1247, 640: 5718, 643: 1247, 1247}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5719, 651: 3684, 2675, 2676, 2674}, // 3260 {1833, 1833, 7: 1833, 50: 1833, 461: 1833, 463: 1833, 469: 1833, 1833, 477: 1833, 1833, 482: 1833, 1833, 1833, 1833, 1833, 488: 1833, 1833, 1833, 502: 1833}, {1832, 1832, 7: 1832, 50: 1832, 461: 1832, 463: 1832, 469: 1832, 1832, 477: 1832, 1832, 482: 1832, 1832, 1832, 1832, 1832, 488: 1832, 1832, 1832, 502: 1832}, {1830, 1830, 7: 1830, 50: 1830, 461: 1830, 463: 1830, 469: 1830, 1830, 477: 1830, 1830, 482: 1830, 1830, 1830, 1830, 1830, 488: 1830, 1830, 1830, 502: 1830}, {1829, 1829, 7: 1829, 50: 1829, 461: 1829, 463: 1829, 469: 1829, 1829, 477: 1829, 1829, 482: 1829, 1829, 1829, 1829, 1829, 488: 1829, 1829, 1829, 502: 1829}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5725, 653: 5724, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5726, 651: 5725, 2675, 2676, 2674}, // 3265 {1827, 1827, 7: 1827, 50: 1827, 461: 1827, 463: 1827, 469: 1827, 1827, 477: 1827, 1827, 482: 1827, 1827, 1827, 1827, 1827, 488: 1827, 1827, 1827, 502: 1827}, {1828, 1828, 7: 1828, 50: 1828, 461: 1828, 463: 1828, 469: 1828, 1828, 477: 1828, 1828, 482: 1828, 1828, 1828, 1828, 1828, 488: 1828, 1828, 1828, 502: 1828}, @@ -9943,143 +9938,143 @@ var ( {992, 992}, {1002, 1002}, // 3270 - {70: 5733, 223: 5732}, + {70: 5734, 223: 5733}, {996, 996}, - {848: 5731}, + {846: 5732}, {995, 995}, - {998, 998, 70: 5738}, + {998, 998, 70: 5739}, // 3275 - {223: 5734}, - {997, 997, 70: 5736, 848: 5735}, + {223: 5735}, + {997, 997, 70: 5737, 846: 5736}, {1000, 1000}, - {848: 5737}, + {846: 5738}, {999, 999}, // 3280 - {848: 5739}, + {846: 5740}, {1001, 1001}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5741, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5742, 2675, 2676, 2674}, {1006, 1006}, - {1010, 1010, 470: 5743}, + {1010, 1010, 470: 5744}, // 3285 - {557: 3250, 700: 5745, 1305: 5744}, - {1009, 1009, 7: 5746}, + {557: 3249, 698: 5746, 1303: 5745}, + {1009, 1009, 7: 5747}, {1008, 1008, 7: 1008}, - {557: 3250, 700: 5747}, + {557: 3249, 698: 5748}, {1007, 1007, 7: 1007}, // 3290 - {486: 5749}, - {464: 5751, 557: 3250, 700: 5752, 1255: 5750}, + {486: 5750}, + {464: 5752, 557: 3249, 698: 5753, 1253: 5751}, {1013, 1013}, {1012, 1012}, {1011, 1011}, // 3295 - {2: 1323, 1323, 1323, 1323, 1323, 8: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 51: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 484: 5754, 1047: 5755}, + {2: 1323, 1323, 1323, 1323, 1323, 8: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 51: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 484: 5755, 1045: 5756}, {2: 1322, 1322, 1322, 1322, 1322, 8: 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 51: 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322, 1322}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5756}, - {152: 894, 462: 894, 894, 480: 5259, 491: 894, 501: 894, 559: 894, 636: 894, 847: 5757}, - {152: 5765, 462: 5758, 2506, 491: 5766, 501: 5764, 559: 2504, 636: 2500, 699: 5763, 741: 5761, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 5762, 5760, 3777, 961: 5759, 1046: 5767}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5757}, + {152: 894, 462: 894, 894, 480: 5260, 491: 894, 493: 894, 559: 894, 634: 894, 845: 5758}, + {152: 5766, 462: 5759, 2507, 491: 5767, 493: 5765, 559: 2505, 634: 2501, 697: 5764, 739: 5762, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 5763, 5761, 3778, 959: 5760, 1044: 5768}, // 3300 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 2262, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 2507, 2506, 491: 2505, 559: 2504, 636: 2500, 653: 4038, 2674, 2675, 2673, 699: 5630, 733: 4039, 741: 3778, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 3780, 3779, 3777, 796: 4932, 985: 5779}, - {462: 3794, 827: 5776, 959: 5775}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 2263, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 2508, 2507, 491: 2506, 559: 2505, 634: 2501, 651: 4039, 2675, 2676, 2674, 697: 5631, 732: 4040, 739: 3779, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 3781, 3780, 3778, 794: 4933, 983: 5780}, + {462: 3795, 825: 5777, 957: 5776}, {1315, 1315, 461: 1315, 470: 1315}, {1314, 1314, 461: 1314, 469: 787, 1314, 477: 787, 787}, {1313, 1313, 461: 1313, 470: 1313}, // 3305 - {1312, 1312, 461: 1312, 469: 786, 1312, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, - {1298, 1298, 2910, 2758, 2794, 2912, 2685, 1298, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 1298, 470: 1298, 653: 4038, 2674, 2675, 2673, 733: 5769, 989: 5770, 1170: 5768}, + {1312, 1312, 461: 1312, 469: 786, 1312, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, + {1298, 1298, 2911, 2759, 2795, 2913, 2686, 1298, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 1298, 470: 1298, 651: 4039, 2675, 2676, 2674, 732: 5770, 987: 5771, 1169: 5769}, {462: 1310}, - {462: 1309, 564: 3793, 899: 3792, 960: 3791}, + {462: 1309, 564: 3794, 897: 3793, 958: 3792}, {1293, 1293, 470: 1293}, // 3310 - {1311, 1311, 7: 5773, 461: 1311, 470: 1311}, - {487: 5771}, + {1311, 1311, 7: 5774, 461: 1311, 470: 1311}, + {487: 5772}, {1297, 1297, 7: 1297, 461: 1297, 470: 1297}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3800, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3796, 786: 5772}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3801, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3797, 784: 5773}, {1299, 1299, 7: 1299, 461: 1299, 470: 1299}, // 3315 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 5769, 989: 5774}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 5770, 987: 5775}, {1296, 1296, 7: 1296, 461: 1296, 470: 1296}, - {1316, 1316, 7: 5777, 461: 1316, 470: 1316}, + {1316, 1316, 7: 5778, 461: 1316, 470: 1316}, {1308, 1308, 7: 1308, 461: 1308, 470: 1308}, - {462: 3794, 827: 5778}, + {462: 3795, 825: 5779}, // 3320 {1307, 1307, 7: 1307, 461: 1307, 470: 1307}, - {50: 5780}, - {152: 5765, 462: 2507, 2506, 491: 5766, 559: 2504, 636: 2500, 699: 5785, 741: 5783, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 5784, 5782, 3777, 961: 5781}, - {462: 3794, 827: 5776, 959: 5786}, + {50: 5781}, + {152: 5766, 462: 2508, 2507, 491: 5767, 559: 2505, 634: 2501, 697: 5786, 739: 5784, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 5785, 5783, 3778, 959: 5782}, + {462: 3795, 825: 5777, 957: 5787}, {1320, 1320, 461: 1320, 470: 1320}, // 3325 {1319, 1319, 461: 1319, 469: 787, 1319, 477: 787, 787}, {1318, 1318, 461: 1318, 470: 1318}, - {1317, 1317, 461: 1317, 469: 786, 1317, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, - {1321, 1321, 7: 5777, 461: 1321, 470: 1321}, - {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 484: 1027, 738: 5197, 5196, 5195, 825: 5198, 873: 5788}, + {1317, 1317, 461: 1317, 469: 786, 1317, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, + {1321, 1321, 7: 5778, 461: 1321, 470: 1321}, + {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 484: 1027, 736: 5198, 5197, 5196, 823: 5199, 871: 5789}, // 3330 - {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4085, 484: 1813, 842: 5789}, - {2: 1323, 1323, 1323, 1323, 1323, 8: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 51: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 484: 5754, 1047: 5790}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5791}, - {152: 894, 462: 894, 894, 480: 5259, 491: 894, 501: 894, 559: 894, 636: 894, 847: 5792}, - {152: 5765, 462: 5758, 2506, 491: 5766, 501: 5764, 559: 2504, 636: 2500, 699: 5763, 741: 5761, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 5762, 5760, 3777, 961: 5759, 1046: 5793}, + {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4086, 484: 1813, 840: 5790}, + {2: 1323, 1323, 1323, 1323, 1323, 8: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 51: 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 1323, 484: 5755, 1045: 5791}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5792}, + {152: 894, 462: 894, 894, 480: 5260, 491: 894, 493: 894, 559: 894, 634: 894, 845: 5793}, + {152: 5766, 462: 5759, 2507, 491: 5767, 493: 5765, 559: 2505, 634: 2501, 697: 5764, 739: 5762, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 5763, 5761, 3778, 959: 5760, 1044: 5794}, // 3335 - {1295, 1295, 461: 5795, 470: 1295, 1234: 5794}, + {1295, 1295, 461: 5796, 470: 1295, 1232: 5795}, {1324, 1324, 470: 1324}, - {194: 5796}, - {566: 5797}, - {644: 5798}, + {195: 5797}, + {566: 5798}, + {642: 5799}, // 3340 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 5313, 878: 5314, 912: 5799}, - {1294, 1294, 7: 5316, 470: 1294}, - {1328, 1328, 462: 5808, 642: 1789}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 5314, 876: 5315, 910: 5800}, + {1294, 1294, 7: 5317, 470: 1294}, + {1328, 1328, 462: 5809, 640: 1789}, {1329, 1329}, - {642: 5803}, + {640: 5804}, // 3345 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5804, 2674, 2675, 2673}, - {1327, 1327, 462: 5805}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 5806}, - {50: 5807}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5805, 2675, 2676, 2674}, + {1327, 1327, 462: 5806}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 5807}, + {50: 5808}, {1325, 1325}, // 3350 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 1876, 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 3487, 765: 3649, 813: 5809}, - {50: 5810}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 1876, 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 3488, 763: 3650, 811: 5810}, + {50: 5811}, {1326, 1326}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5962, 2674, 2675, 2673}, - {581, 581, 489: 5959, 510: 5958, 1270: 5957}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5963, 2675, 2676, 2674}, + {581, 581, 489: 5960, 510: 5959, 1268: 5958}, // 3355 - {16: 5945, 99: 5942, 132: 5947, 162: 5946, 186: 5944, 559: 5941, 572: 5943}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 5930, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5931}, - {653, 653, 482: 5925}, - {125: 5924}, - {98: 3841, 107: 3840, 110: 5919, 207: 5918, 821: 5920}, + {16: 5946, 99: 5943, 132: 5948, 162: 5947, 187: 5945, 559: 5942, 572: 5944}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 5931, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5932}, + {653, 653, 482: 5926}, + {125: 5925}, + {98: 3842, 107: 3841, 110: 5920, 207: 5919, 819: 5921}, // 3360 {649, 649}, - {641, 641, 177: 5900, 219: 5901, 229: 5902, 232: 5899, 251: 5904, 261: 5903, 275: 5906, 280: 5905, 482: 641, 641, 488: 641, 641: 5907, 1109: 5898, 1273: 5897, 5896}, + {641, 641, 178: 5901, 219: 5902, 229: 5903, 232: 5900, 251: 5905, 261: 5904, 276: 5907, 281: 5906, 482: 641, 641, 488: 641, 639: 5908, 1108: 5899, 1271: 5898, 5897}, {647, 647}, {646, 646}, - {583, 583, 254: 5887, 482: 5886, 489: 583, 510: 583}, + {583, 583, 254: 5888, 482: 5887, 489: 583, 510: 583}, // 3365 - {486: 624, 533: 624}, - {486: 623, 533: 623}, - {486: 622, 533: 622}, + {486: 624, 532: 624}, + {486: 623, 532: 623}, + {486: 622, 532: 622}, {619, 619, 489: 619, 510: 619}, {618, 618, 489: 618, 510: 618}, // 3370 {617, 617, 489: 617, 510: 617}, {616, 616, 489: 616, 510: 616}, - {110: 5884}, - {486: 5860, 533: 5861, 793: 5879}, - {98: 571, 107: 571, 199: 5858, 1071: 5873}, + {110: 5885}, + {486: 5861, 532: 5862, 791: 5880}, + {98: 571, 107: 571, 200: 5859, 1070: 5874}, // 3375 {608, 608, 489: 608, 510: 608}, {607, 607, 489: 607, 510: 607}, - {125: 5871, 138: 5872, 191: 5870}, + {125: 5872, 138: 5873, 192: 5871}, {603, 603, 489: 603, 510: 603}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5869}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5870}, // 3380 + {125: 5869}, {125: 5868}, {125: 5867}, {125: 5866}, - {125: 5865}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5862}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5863}, // 3385 {596, 596, 489: 596, 510: 596}, {595, 595, 489: 595, 510: 595}, @@ -10091,13 +10086,13 @@ var ( {590, 590, 489: 590, 510: 590}, {589, 589, 489: 589, 510: 589}, {588, 588, 489: 588, 510: 588}, - {125: 5859}, + {125: 5860}, // 3395 {586, 586, 489: 586, 510: 586}, {585, 585, 489: 585, 510: 585}, {584, 584, 489: 584, 510: 584}, - {125: 577, 138: 577, 191: 577}, - {125: 576, 138: 576, 154: 576, 191: 576}, + {125: 577, 138: 577, 192: 577}, + {125: 576, 138: 576, 154: 576, 192: 576}, // 3400 {98: 570, 107: 570, 110: 570, 207: 570}, {587, 587, 489: 587, 510: 587}, @@ -10105,7 +10100,7 @@ var ( {2: 620, 620, 620, 620, 620, 8: 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 51: 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620, 620}, {597, 597, 489: 597, 510: 597}, // 3405 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 5864}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 5865}, {568, 568, 489: 568, 510: 568}, {598, 598, 489: 598, 510: 598}, {599, 599, 489: 599, 510: 599}, @@ -10117,43 +10112,43 @@ var ( {605, 605, 489: 605, 510: 605}, {604, 604, 489: 604, 510: 604}, // 3415 - {98: 3841, 107: 3840, 821: 5874}, - {486: 5860, 533: 5861, 793: 5876, 1111: 5875}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5878}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5877}, - {567, 567, 486: 567, 489: 567, 510: 567, 533: 567}, + {98: 3842, 107: 3841, 819: 5875}, + {486: 5861, 532: 5862, 791: 5877, 1110: 5876}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5879}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5878}, + {567, 567, 486: 567, 489: 567, 510: 567, 532: 567}, // 3420 {609, 609, 489: 609, 510: 609}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5880, 2674, 2675, 2673, 727: 5881}, - {1025, 1025, 486: 5860, 489: 1025, 510: 1025, 533: 5861, 642: 3823, 793: 5882}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5881, 2675, 2676, 2674, 725: 5882}, + {1025, 1025, 486: 5861, 489: 1025, 510: 1025, 532: 5862, 640: 3824, 791: 5883}, {612, 612, 489: 612, 510: 612}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5883, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5884, 2675, 2676, 2674}, // 3425 {611, 611, 489: 611, 510: 611}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5885}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5886}, {614, 614, 489: 614, 510: 614}, - {559: 5891, 572: 5888, 837: 5890, 1271: 5889}, + {559: 5892, 572: 5889, 835: 5891, 1269: 5890}, {582, 582, 489: 582, 510: 582}, // 3430 - {2: 2020, 2020, 2020, 2020, 2020, 8: 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 51: 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 468: 2020, 473: 2020, 494: 2020, 512: 2020, 555: 2020, 637: 2020}, + {2: 2020, 2020, 2020, 2020, 2020, 8: 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 51: 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 2020, 468: 2020, 473: 2020, 493: 2020, 495: 2020, 512: 2020, 555: 2020, 635: 2020}, {645, 645}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 5895}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5892}, - {643, 643, 480: 5893}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 5896}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5893}, + {643, 643, 480: 5894}, // 3435 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5894, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5895, 2675, 2676, 2674}, {642, 642}, {644, 644}, - {628, 628, 482: 5914, 628, 488: 628, 1272: 5913}, - {640, 640, 7: 5911, 482: 640, 640, 488: 640}, + {628, 628, 482: 5915, 628, 488: 628, 1270: 5914}, + {640, 640, 7: 5912, 482: 640, 640, 488: 640}, // 3440 {639, 639, 7: 639, 482: 639, 639, 488: 639}, {637, 637, 7: 637, 482: 637, 637, 488: 637}, {636, 636, 7: 636, 482: 636, 636, 488: 636}, - {336: 5910}, - {377: 5909}, + {338: 5911}, + {378: 5910}, // 3445 - {325: 5908}, + {327: 5909}, {632, 632, 7: 632, 482: 632, 632, 488: 632}, {631, 631, 7: 631, 482: 631, 631, 488: 631}, {630, 630, 7: 630, 482: 630, 630, 488: 630}, @@ -10162,229 +10157,229 @@ var ( {633, 633, 7: 633, 482: 633, 633, 488: 633}, {634, 634, 7: 634, 482: 634, 634, 488: 634}, {635, 635, 7: 635, 482: 635, 635, 488: 635}, - {177: 5900, 219: 5901, 229: 5902, 232: 5899, 251: 5904, 261: 5903, 275: 5906, 280: 5905, 641: 5907, 1109: 5912}, + {178: 5901, 219: 5902, 229: 5903, 232: 5900, 251: 5905, 261: 5904, 276: 5907, 281: 5906, 639: 5908, 1108: 5913}, {638, 638, 7: 638, 482: 638, 638, 488: 638}, // 3455 - {841, 841, 483: 2640, 488: 2641, 757: 2642, 816: 5917}, - {143: 5915}, - {496: 2648, 725: 3948, 751: 5916}, + {841, 841, 483: 2641, 488: 2642, 755: 2643, 814: 5918}, + {143: 5916}, + {497: 2649, 721: 3949, 749: 5917}, {627, 627, 483: 627, 488: 627}, {648, 648}, // 3460 {650, 650}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5923}, - {486: 5860, 533: 5861, 793: 5876, 1111: 5921}, - {569, 569, 486: 5860, 489: 569, 510: 569, 533: 5861, 793: 5863, 829: 5922}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5924}, + {486: 5861, 532: 5862, 791: 5877, 1110: 5922}, + {569, 569, 486: 5861, 489: 569, 510: 569, 532: 5862, 791: 5864, 827: 5923}, {610, 610, 489: 610, 510: 610}, // 3465 {615, 615, 489: 615, 510: 615}, {651, 651}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5926}, - {626, 626, 470: 5928, 1306: 5927}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5927}, + {626, 626, 470: 5929, 1304: 5928}, {652, 652}, // 3470 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 5929}, - {625, 625, 7: 5525}, - {569, 569, 86: 1693, 157: 1693, 480: 1693, 486: 5860, 489: 569, 510: 569, 533: 5861, 639: 1693, 642: 1693, 793: 5863, 829: 5940}, - {86: 894, 157: 5933, 480: 5259, 639: 894, 847: 5932}, - {86: 5934, 639: 5935}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 5930}, + {625, 625, 7: 5526}, + {569, 569, 86: 1693, 157: 1693, 480: 1693, 486: 5861, 489: 569, 510: 569, 532: 5862, 637: 1693, 640: 1693, 791: 5864, 827: 5941}, + {86: 894, 157: 5934, 480: 5260, 637: 894, 845: 5933}, + {86: 5935, 637: 5936}, // 3475 {655, 655}, - {237, 237, 489: 2634, 778: 2635, 5939}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5936, 2674, 2675, 2673}, - {86: 5937}, - {237, 237, 489: 2634, 778: 2635, 5938}, + {237, 237, 489: 2635, 776: 2636, 5940}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5937, 2675, 2676, 2674}, + {86: 5938}, + {237, 237, 489: 2635, 776: 2636, 5939}, // 3480 {654, 654}, {656, 656}, {613, 613, 489: 613, 510: 613}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5956}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5955}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5957}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5956}, // 3485 - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 5953}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 5952}, - {158: 5950}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5949}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5948, 2674, 2675, 2673}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 5954}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 5953}, + {158: 5951}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5950}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5949, 2675, 2676, 2674}, // 3490 {657, 657}, {658, 658}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4862, 2674, 2675, 2673, 872: 5951}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4863, 2675, 2676, 2674, 870: 5952}, {659, 659}, {660, 660}, // 3495 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 5954}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 5955}, {661, 661}, {662, 662}, {663, 663}, {664, 664}, // 3500 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 3250, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 5961, 3153, 3236, 3152, 3149}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 2751, 2699, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 2780, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 2678, 2694, 2837, 2928, 2785, 2712, 2729, 2856, 2939, 2772, 2741, 2850, 2851, 2846, 2806, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 2787, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 2791, 2672, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 2710, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 2776, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 2777, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 2845, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 2663, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 2793, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 2735, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 2664, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 2688, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3062, 3063, 3113, 3112, 2965, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 2827, 2844, 2966, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3092, 3093, 3103, 3088, 3089, 3090, 3121, 2788, 462: 3160, 464: 3140, 3158, 2667, 468: 3168, 471: 3173, 3177, 474: 3156, 3157, 3195, 481: 3131, 491: 3169, 494: 3193, 3176, 3135, 532: 3164, 555: 3171, 3194, 2665, 3178, 560: 3130, 3132, 3134, 3133, 3161, 3138, 567: 3151, 3163, 3139, 3172, 572: 3170, 3162, 575: 3167, 577: 3238, 3174, 3183, 3184, 3185, 3137, 3154, 3155, 3208, 3211, 3212, 3213, 3214, 3215, 3165, 3216, 3191, 3196, 3206, 3207, 3200, 3217, 3218, 3219, 3201, 3221, 3222, 3209, 3202, 3220, 3197, 3205, 3203, 3189, 3223, 3224, 3166, 3228, 3179, 3180, 3182, 3227, 3233, 3232, 3234, 3231, 3235, 3230, 3229, 3226, 3175, 3225, 3181, 3186, 3187, 638: 2668, 653: 3144, 2674, 2675, 2673, 699: 3159, 3237, 3145, 3150, 3136, 3210, 3148, 3146, 3147, 3188, 3199, 3198, 3192, 3190, 3204, 3143, 3153, 3236, 3152, 3149, 2671, 2670, 2669, 5960}, - {579, 579, 493: 3247, 497: 3245, 3246, 3244, 3242, 723: 3243, 3241}, - {580, 580, 473: 3251, 571: 3252}, - {1912, 1912, 200: 5964, 559: 1912, 1237: 5963}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 3249, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 5962, 3154, 3235, 3153, 3150}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 2752, 2700, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 2781, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 2679, 2695, 2838, 2929, 2786, 2713, 2730, 2857, 2940, 2773, 2742, 2851, 2852, 2847, 2807, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 2788, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 2792, 2673, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 2711, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 2777, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 2778, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 2846, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 2664, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 2794, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 2736, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 2665, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3048, 2790, 3049, 3050, 2689, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3063, 3064, 3114, 3113, 2966, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 2828, 2845, 2967, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3081, 3082, 3083, 2841, 3036, 3093, 3094, 3104, 3089, 3090, 3091, 3122, 2789, 462: 3161, 464: 3141, 3159, 2668, 468: 3169, 471: 3174, 3178, 474: 3157, 3158, 3196, 481: 3132, 491: 3170, 495: 3194, 3177, 3136, 534: 3165, 555: 3172, 3195, 2666, 3179, 560: 3131, 3133, 3135, 3134, 3162, 3139, 567: 3152, 3164, 3140, 3173, 572: 3171, 3163, 575: 3168, 577: 3237, 3175, 3184, 3185, 3186, 3138, 3155, 3156, 3210, 3211, 3212, 3213, 3214, 3166, 3215, 3192, 3197, 3207, 3208, 3201, 3216, 3217, 3218, 3202, 3220, 3221, 3203, 3219, 3198, 3206, 3204, 3190, 3222, 3223, 3167, 3227, 3180, 3181, 3183, 3226, 3232, 3231, 3233, 3230, 3234, 3229, 3228, 3225, 3176, 3224, 3182, 3187, 3188, 636: 2669, 651: 3145, 2675, 2676, 2674, 697: 3160, 3236, 3146, 3151, 3137, 3209, 3149, 3147, 3148, 3189, 3200, 3199, 3193, 3191, 3205, 3144, 3154, 3235, 3153, 3150, 2672, 2671, 2670, 5961}, + {579, 579, 494: 3246, 498: 3244, 3245, 3243, 3241, 722: 3242, 3240}, + {580, 580, 473: 3250, 571: 3251}, + {1912, 1912, 201: 5965, 559: 1912, 1235: 5964}, // 3505 - {545, 545, 559: 5966, 953: 5965}, + {545, 545, 559: 5967, 951: 5966}, {1911, 1911, 559: 1911}, {1917, 1917}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 5967}, - {544, 544, 7: 3821}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 5968}, + {544, 544, 7: 3822}, // 3510 - {2: 1914, 1914, 1914, 1914, 1914, 8: 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 51: 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 555: 5970, 1203: 5969}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5973, 2674, 2675, 2673}, - {465: 3984, 3983, 798: 5971}, - {184: 5972}, + {2: 1914, 1914, 1914, 1914, 1914, 8: 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 51: 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 1914, 555: 5971, 1202: 5970}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5974, 2675, 2676, 2674}, + {465: 3985, 3984, 796: 5972}, + {185: 5973}, {2: 1913, 1913, 1913, 1913, 1913, 8: 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 51: 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913, 1913}, // 3515 {1920, 1920}, - {2: 1916, 1916, 1916, 1916, 1916, 8: 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 51: 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 555: 5976, 1204: 5975}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5978, 2674, 2675, 2673}, - {184: 5977}, + {2: 1916, 1916, 1916, 1916, 1916, 8: 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 51: 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 1916, 555: 5977, 1203: 5976}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5979, 2675, 2676, 2674}, + {185: 5978}, {2: 1915, 1915, 1915, 1915, 1915, 8: 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 51: 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915, 1915}, // 3520 {1921, 1921}, - {496: 2648, 725: 5980}, + {497: 2649, 721: 5981}, {1923, 1923}, - {486: 5990}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 5985, 653: 5190, 2674, 2675, 2673, 803: 5987, 1179: 5986}, + {486: 5991}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 5986, 651: 5191, 2675, 2676, 2674, 801: 5988, 1178: 5987}, // 3525 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 5984}, - {7: 3821, 486: 1970, 640: 1970}, - {486: 1972, 640: 1972}, - {7: 5988, 486: 1971, 640: 1971}, - {7: 1969, 486: 1969, 640: 1969}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 5985}, + {7: 3822, 486: 1970, 638: 1970}, + {486: 1972, 638: 1972}, + {7: 5989, 486: 1971, 638: 1971}, + {7: 1969, 486: 1969, 638: 1969}, // 3530 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 5989}, - {7: 1968, 486: 1968, 640: 1968}, - {464: 5991}, - {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 643: 1967, 885: 5992}, - {1973, 1973, 15: 6019, 47: 5995, 49: 6015, 51: 6008, 5998, 5994, 6002, 6006, 6018, 6001, 6007, 6005, 6003, 6016, 6009, 5997, 6017, 5996, 5999, 6000, 6004, 461: 6010, 643: 6020, 881: 6012, 6011, 6014, 5993, 886: 6013}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 5990}, + {7: 1968, 486: 1968, 638: 1968}, + {464: 5992}, + {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 641: 1967, 883: 5993}, + {1973, 1973, 15: 6020, 47: 5996, 49: 6016, 51: 6009, 5999, 5995, 6003, 6007, 6019, 6002, 6008, 6006, 6004, 6017, 6010, 5998, 6018, 5997, 6000, 6001, 6005, 461: 6011, 641: 6021, 879: 6013, 6012, 6015, 5994, 884: 6014}, // 3535 - {1966, 1966, 15: 1966, 47: 1966, 49: 1966, 51: 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 461: 1966, 643: 1966}, - {487: 1965, 496: 1965}, - {487: 1964, 496: 1964}, - {487: 1963, 496: 1963, 560: 1963, 1963}, - {487: 1962, 496: 1962, 560: 1962, 1962}, + {1966, 1966, 15: 1966, 47: 1966, 49: 1966, 51: 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 1966, 461: 1966, 641: 1966}, + {487: 1965, 497: 1965}, + {487: 1964, 497: 1964}, + {487: 1963, 497: 1963, 560: 1963, 1963}, + {487: 1962, 497: 1962, 560: 1962, 1962}, // 3540 - {487: 1961, 496: 1961, 560: 1961, 1961}, - {487: 1960, 496: 1960, 560: 1960, 1960}, - {487: 1959, 496: 1959, 560: 1959, 1959}, - {487: 1958, 496: 1958, 560: 1958, 1958}, - {487: 1957, 496: 1957, 560: 1957, 1957}, + {487: 1961, 497: 1961, 560: 1961, 1961}, + {487: 1960, 497: 1960, 560: 1960, 1960}, + {487: 1959, 497: 1959, 560: 1959, 1959}, + {487: 1958, 497: 1958, 560: 1958, 1958}, + {487: 1957, 497: 1957, 560: 1957, 1957}, // 3545 - {487: 1956, 496: 1956, 560: 1956, 1956}, + {487: 1956, 497: 1956, 560: 1956, 1956}, {464: 1955, 487: 1955}, {464: 1954, 487: 1954}, {464: 1953, 487: 1953}, {464: 1952, 487: 1952}, // 3550 - {2: 1951, 1951, 1951, 1951, 1951, 8: 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 51: 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 464: 1951, 479: 1951, 487: 1951, 495: 1951}, - {2: 1950, 1950, 1950, 1950, 1950, 8: 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 51: 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 464: 1950, 479: 1950, 487: 1950, 495: 1950}, - {194: 6060}, - {487: 4172, 496: 1999, 726: 6058}, - {487: 4172, 496: 1999, 560: 1999, 1999, 726: 6056}, + {2: 1951, 1951, 1951, 1951, 1951, 8: 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 51: 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 1951, 464: 1951, 479: 1951, 487: 1951, 496: 1951}, + {2: 1950, 1950, 1950, 1950, 1950, 8: 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 51: 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 1950, 464: 1950, 479: 1950, 487: 1950, 496: 1950}, + {195: 6061}, + {487: 4173, 497: 1999, 724: 6059}, + {487: 4173, 497: 1999, 560: 1999, 1999, 724: 6057}, // 3555 - {464: 1999, 487: 4172, 726: 6054}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 479: 1999, 487: 4172, 495: 1999, 726: 6049}, - {464: 1999, 487: 4172, 496: 1999, 726: 6044}, - {464: 1999, 487: 4172, 496: 1999, 726: 6041}, - {487: 4172, 496: 1999, 726: 6036}, + {464: 1999, 487: 4173, 724: 6055}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 479: 1999, 487: 4173, 496: 1999, 724: 6050}, + {464: 1999, 487: 4173, 497: 1999, 724: 6045}, + {464: 1999, 487: 4173, 497: 1999, 724: 6042}, + {487: 4173, 497: 1999, 724: 6037}, // 3560 - {98: 1999, 107: 1999, 487: 4172, 496: 1999, 726: 6033}, - {178: 1999, 1999, 182: 1999, 487: 4172, 496: 1999, 560: 1999, 1999, 726: 6030}, - {178: 1999, 1999, 182: 1999, 487: 4172, 496: 1999, 560: 1999, 1999, 726: 6021}, - {178: 6027, 6028, 182: 6029, 496: 2648, 560: 6025, 6026, 725: 6024, 914: 6022, 1074: 6023}, - {1934, 1934, 15: 1934, 47: 1934, 49: 1934, 51: 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 461: 1934, 643: 1934}, + {98: 1999, 107: 1999, 487: 4173, 497: 1999, 724: 6034}, + {179: 1999, 1999, 183: 1999, 487: 4173, 497: 1999, 560: 1999, 1999, 724: 6031}, + {179: 1999, 1999, 183: 1999, 487: 4173, 497: 1999, 560: 1999, 1999, 724: 6022}, + {179: 6028, 6029, 183: 6030, 497: 2649, 560: 6026, 6027, 721: 6025, 912: 6023, 1073: 6024}, + {1934, 1934, 15: 1934, 47: 1934, 49: 1934, 51: 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 1934, 461: 1934, 641: 1934}, // 3565 - {1933, 1933, 15: 1933, 47: 1933, 49: 1933, 51: 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 461: 1933, 643: 1933}, - {1929, 1929, 15: 1929, 47: 1929, 49: 1929, 51: 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 461: 1929, 643: 1929}, - {1928, 1928, 15: 1928, 47: 1928, 49: 1928, 51: 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 461: 1928, 643: 1928}, - {1927, 1927, 15: 1927, 47: 1927, 49: 1927, 51: 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 461: 1927, 643: 1927}, - {1926, 1926, 15: 1926, 47: 1926, 49: 1926, 51: 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 461: 1926, 643: 1926}, + {1933, 1933, 15: 1933, 47: 1933, 49: 1933, 51: 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 1933, 461: 1933, 641: 1933}, + {1929, 1929, 15: 1929, 47: 1929, 49: 1929, 51: 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 1929, 461: 1929, 641: 1929}, + {1928, 1928, 15: 1928, 47: 1928, 49: 1928, 51: 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 1928, 461: 1928, 641: 1928}, + {1927, 1927, 15: 1927, 47: 1927, 49: 1927, 51: 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 1927, 461: 1927, 641: 1927}, + {1926, 1926, 15: 1926, 47: 1926, 49: 1926, 51: 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 1926, 461: 1926, 641: 1926}, // 3570 - {1925, 1925, 15: 1925, 47: 1925, 49: 1925, 51: 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 461: 1925, 643: 1925}, - {1924, 1924, 15: 1924, 47: 1924, 49: 1924, 51: 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 461: 1924, 643: 1924}, - {178: 6027, 6028, 182: 6029, 496: 2648, 560: 6025, 6026, 725: 6024, 914: 6031, 1074: 6032}, - {1936, 1936, 15: 1936, 47: 1936, 49: 1936, 51: 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 461: 1936, 643: 1936}, - {1935, 1935, 15: 1935, 47: 1935, 49: 1935, 51: 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 461: 1935, 643: 1935}, + {1925, 1925, 15: 1925, 47: 1925, 49: 1925, 51: 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 1925, 461: 1925, 641: 1925}, + {1924, 1924, 15: 1924, 47: 1924, 49: 1924, 51: 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 1924, 461: 1924, 641: 1924}, + {179: 6028, 6029, 183: 6030, 497: 2649, 560: 6026, 6027, 721: 6025, 912: 6032, 1073: 6033}, + {1936, 1936, 15: 1936, 47: 1936, 49: 1936, 51: 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 1936, 461: 1936, 641: 1936}, + {1935, 1935, 15: 1935, 47: 1935, 49: 1935, 51: 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 1935, 461: 1935, 641: 1935}, // 3575 - {98: 3841, 107: 3840, 496: 2648, 725: 2647, 734: 6035, 821: 6034}, - {1938, 1938, 15: 1938, 47: 1938, 49: 1938, 51: 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 461: 1938, 643: 1938}, - {1937, 1937, 15: 1937, 47: 1937, 49: 1937, 51: 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 461: 1937, 643: 1937}, - {496: 2648, 725: 2647, 734: 6037}, - {202: 6038}, + {98: 3842, 107: 3841, 497: 2649, 721: 2648, 731: 6036, 819: 6035}, + {1938, 1938, 15: 1938, 47: 1938, 49: 1938, 51: 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 1938, 461: 1938, 641: 1938}, + {1937, 1937, 15: 1937, 47: 1937, 49: 1937, 51: 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 1937, 461: 1937, 641: 1937}, + {497: 2649, 721: 2648, 731: 6038}, + {202: 6039}, // 3580 - {545: 6039}, - {108: 6040}, - {1939, 1939, 15: 1939, 47: 1939, 49: 1939, 51: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 461: 1939, 643: 1939}, - {464: 6042, 496: 2648, 725: 2647, 734: 6043}, - {1941, 1941, 15: 1941, 47: 1941, 49: 1941, 51: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 461: 1941, 643: 1941}, + {545: 6040}, + {108: 6041}, + {1939, 1939, 15: 1939, 47: 1939, 49: 1939, 51: 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 1939, 461: 1939, 641: 1939}, + {464: 6043, 497: 2649, 721: 2648, 731: 6044}, + {1941, 1941, 15: 1941, 47: 1941, 49: 1941, 51: 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 1941, 461: 1941, 641: 1941}, // 3585 - {1940, 1940, 15: 1940, 47: 1940, 49: 1940, 51: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 461: 1940, 643: 1940}, - {464: 6046, 496: 2648, 725: 2647, 734: 6045}, - {1942, 1942, 15: 1942, 47: 1942, 49: 1942, 51: 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 105: 3363, 3359, 108: 3356, 3371, 111: 3358, 3355, 3357, 3361, 3362, 3367, 3366, 3365, 3369, 3370, 3364, 3368, 3360, 461: 1942, 643: 1942, 794: 6047}, - {1943, 1943, 15: 1943, 47: 1943, 49: 1943, 51: 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 461: 1943, 643: 1943}, - {294: 6048}, + {1940, 1940, 15: 1940, 47: 1940, 49: 1940, 51: 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 1940, 461: 1940, 641: 1940}, + {464: 6047, 497: 2649, 721: 2648, 731: 6046}, + {1942, 1942, 15: 1942, 47: 1942, 49: 1942, 51: 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 1942, 105: 3364, 3360, 108: 3357, 3372, 111: 3359, 3356, 3358, 3362, 3363, 3368, 3367, 3366, 3370, 3371, 3365, 3369, 3361, 461: 1942, 641: 1942, 792: 6048}, + {1943, 1943, 15: 1943, 47: 1943, 49: 1943, 51: 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 1943, 461: 1943, 641: 1943}, + {296: 6049}, // 3590 - {1944, 1944, 15: 1944, 47: 1944, 49: 1944, 51: 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 461: 1944, 643: 1944}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 479: 6052, 495: 6053, 653: 3390, 2674, 2675, 2673, 728: 6051, 1287: 6050}, - {1945, 1945, 15: 1945, 47: 1945, 49: 1945, 51: 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 461: 1945, 643: 1945}, - {246, 246, 15: 246, 47: 246, 49: 246, 51: 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 461: 246, 643: 246}, - {245, 245, 15: 245, 47: 245, 49: 245, 51: 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 461: 245, 643: 245}, + {1944, 1944, 15: 1944, 47: 1944, 49: 1944, 51: 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 1944, 461: 1944, 641: 1944}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 479: 6053, 496: 6054, 651: 3391, 2675, 2676, 2674, 726: 6052, 1285: 6051}, + {1945, 1945, 15: 1945, 47: 1945, 49: 1945, 51: 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 1945, 461: 1945, 641: 1945}, + {246, 246, 15: 246, 47: 246, 49: 246, 51: 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 246, 461: 246, 641: 246}, + {245, 245, 15: 245, 47: 245, 49: 245, 51: 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 245, 461: 245, 641: 245}, // 3595 - {244, 244, 15: 244, 47: 244, 49: 244, 51: 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 461: 244, 643: 244}, - {464: 6055}, - {1946, 1946, 15: 1946, 47: 1946, 49: 1946, 51: 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 461: 1946, 643: 1946}, - {496: 2648, 560: 6025, 6026, 725: 6024, 914: 6057}, - {1947, 1947, 15: 1947, 47: 1947, 49: 1947, 51: 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 461: 1947, 643: 1947}, + {244, 244, 15: 244, 47: 244, 49: 244, 51: 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 244, 461: 244, 641: 244}, + {464: 6056}, + {1946, 1946, 15: 1946, 47: 1946, 49: 1946, 51: 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 1946, 461: 1946, 641: 1946}, + {497: 2649, 560: 6026, 6027, 721: 6025, 912: 6058}, + {1947, 1947, 15: 1947, 47: 1947, 49: 1947, 51: 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 1947, 461: 1947, 641: 1947}, // 3600 - {496: 2648, 725: 2647, 734: 6059}, - {1948, 1948, 15: 1948, 47: 1948, 49: 1948, 51: 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 461: 1948, 643: 1948}, - {2: 1949, 1949, 1949, 1949, 1949, 8: 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 51: 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 464: 1949, 479: 1949, 487: 1949, 495: 1949}, - {640: 6062}, - {464: 6063}, + {497: 2649, 721: 2648, 731: 6060}, + {1948, 1948, 15: 1948, 47: 1948, 49: 1948, 51: 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 1948, 461: 1948, 641: 1948}, + {2: 1949, 1949, 1949, 1949, 1949, 8: 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 51: 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 1949, 464: 1949, 479: 1949, 487: 1949, 496: 1949}, + {638: 6063}, + {464: 6064}, // 3605 - {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 643: 1967, 885: 6064}, - {1974, 1974, 15: 6019, 47: 5995, 49: 6015, 51: 6008, 5998, 5994, 6002, 6006, 6018, 6001, 6007, 6005, 6003, 6016, 6009, 5997, 6017, 5996, 5999, 6000, 6004, 461: 6010, 643: 6020, 881: 6012, 6011, 6014, 5993, 886: 6013}, - {1989, 1989, 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 6090}, + {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 641: 1967, 883: 6065}, + {1974, 1974, 15: 6020, 47: 5996, 49: 6016, 51: 6009, 5999, 5995, 6003, 6007, 6019, 6002, 6008, 6006, 6004, 6017, 6010, 5998, 6018, 5997, 6000, 6001, 6005, 461: 6011, 641: 6021, 879: 6013, 6012, 6015, 5994, 884: 6014}, + {1989, 1989, 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 6091}, {1987, 1987}, - {26: 6088}, + {26: 6089}, // 3610 - {1726, 1726, 1726, 1726, 1726, 1726, 1726, 8: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 51: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 487: 6071, 642: 1726}, - {462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 652: 2620, 699: 2621, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 6070}, + {1726, 1726, 1726, 1726, 1726, 1726, 1726, 8: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 51: 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 1726, 487: 6072, 640: 1726}, + {462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 650: 2621, 697: 2622, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 6071}, {1981, 1981}, - {175: 6075, 300: 6078, 316: 6077, 388: 6074, 393: 6079, 464: 6072, 564: 6076, 1184: 6073}, - {462: 2507, 2506, 482: 6084, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 652: 2620, 699: 2621, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 6085}, + {175: 6076, 302: 6079, 318: 6078, 388: 6075, 393: 6080, 464: 6073, 564: 6077, 1183: 6074}, + {462: 2508, 2507, 482: 6085, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 650: 2621, 697: 2622, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 6086}, // 3615 - {462: 2507, 2506, 482: 6080, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 652: 2620, 699: 2621, 732: 2474, 741: 2622, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2628, 2627, 2477, 763: 2603, 2475, 768: 2625, 2626, 771: 2624, 781: 2476, 785: 2623, 810: 2629, 839: 6081}, - {462: 1980, 1980, 482: 1980, 491: 1980, 495: 1980, 556: 1980, 559: 1980, 636: 1980, 644: 1980, 652: 1980, 732: 1980}, - {462: 1979, 1979, 482: 1979, 491: 1979, 495: 1979, 556: 1979, 559: 1979, 636: 1979, 644: 1979, 652: 1979, 732: 1979}, - {462: 1978, 1978, 482: 1978, 491: 1978, 495: 1978, 556: 1978, 559: 1978, 636: 1978, 644: 1978, 652: 1978, 732: 1978}, - {462: 1977, 1977, 482: 1977, 491: 1977, 495: 1977, 556: 1977, 559: 1977, 636: 1977, 644: 1977, 652: 1977, 732: 1977}, + {462: 2508, 2507, 482: 6081, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 650: 2621, 697: 2622, 730: 2475, 739: 2623, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2629, 2628, 2478, 761: 2604, 2476, 766: 2626, 2627, 769: 2625, 779: 2477, 783: 2624, 808: 2630, 837: 6082}, + {462: 1980, 1980, 482: 1980, 491: 1980, 496: 1980, 556: 1980, 559: 1980, 634: 1980, 642: 1980, 650: 1980, 730: 1980}, + {462: 1979, 1979, 482: 1979, 491: 1979, 496: 1979, 556: 1979, 559: 1979, 634: 1979, 642: 1979, 650: 1979, 730: 1979}, + {462: 1978, 1978, 482: 1978, 491: 1978, 496: 1978, 556: 1978, 559: 1978, 634: 1978, 642: 1978, 650: 1978, 730: 1978}, + {462: 1977, 1977, 482: 1977, 491: 1977, 496: 1977, 556: 1977, 559: 1977, 634: 1977, 642: 1977, 650: 1977, 730: 1977}, // 3620 - {462: 1976, 1976, 482: 1976, 491: 1976, 495: 1976, 556: 1976, 559: 1976, 636: 1976, 644: 1976, 652: 1976, 732: 1976}, - {462: 1975, 1975, 482: 1975, 491: 1975, 495: 1975, 556: 1975, 559: 1975, 636: 1975, 644: 1975, 652: 1975, 732: 1975}, - {26: 6082}, + {462: 1976, 1976, 482: 1976, 491: 1976, 496: 1976, 556: 1976, 559: 1976, 634: 1976, 642: 1976, 650: 1976, 730: 1976}, + {462: 1975, 1975, 482: 1975, 491: 1975, 496: 1975, 556: 1975, 559: 1975, 634: 1975, 642: 1975, 650: 1975, 730: 1975}, + {26: 6083}, {1982, 1982}, - {496: 2648, 725: 6083}, + {497: 2649, 721: 6084}, // 3625 {1983, 1983}, - {26: 6086}, + {26: 6087}, {1984, 1984}, - {496: 2648, 725: 6087}, + {497: 2649, 721: 6088}, {1985, 1985}, // 3630 - {496: 2648, 725: 6089}, + {497: 2649, 721: 6090}, {1986, 1986}, {1988, 1988}, {1996, 1996}, - {487: 6114}, + {487: 6115}, // 3635 - {71: 2468, 145: 2470, 151: 2496, 153: 2467, 380: 6110, 462: 2507, 2506, 491: 2505, 495: 2491, 501: 6096, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6094, 732: 2474, 741: 6095, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6102, 6101, 2477, 763: 2603, 2475, 768: 6099, 6100, 771: 6098, 781: 2476, 785: 6097, 797: 6108, 832: 6104, 835: 6105, 846: 6103, 850: 6106, 6107, 907: 6109}, - {449, 449, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {71: 2469, 145: 2471, 151: 2497, 153: 2468, 381: 6111, 462: 2508, 2507, 491: 2506, 493: 6097, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6095, 730: 2475, 739: 6096, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6103, 6102, 2478, 761: 2604, 2476, 766: 6100, 6101, 769: 6099, 779: 2477, 783: 6098, 795: 6109, 830: 6105, 833: 6106, 844: 6104, 848: 6107, 6108, 905: 6110}, + {449, 449, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, {451, 451, 469: 787, 477: 787, 787}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 5487, 5482, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 5485, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 5484, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 5489, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 5483, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 5490, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 5486, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 494: 3734, 557: 5496, 577: 5495, 637: 3732, 653: 5493, 2674, 2675, 2673, 761: 5497, 818: 5494, 962: 5498, 1137: 5491}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 5488, 5483, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 5486, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 5485, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 5490, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 5484, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 5491, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 5487, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 495: 3735, 557: 5497, 577: 5496, 635: 3733, 651: 5494, 2675, 2676, 2674, 759: 5498, 816: 5495, 960: 5499, 1136: 5492}, {456, 456}, // 3640 {455, 455}, @@ -10399,89 +10394,89 @@ var ( {445, 445}, {444, 444}, // 3650 - {20: 4994}, + {21: 4995}, {1994, 1994}, - {487: 6111}, - {464: 6112}, - {71: 2468, 145: 2470, 151: 2496, 153: 2467, 462: 2507, 2506, 491: 2505, 495: 2491, 501: 6096, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6094, 732: 2474, 741: 6095, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6102, 6101, 2477, 763: 2603, 2475, 768: 6099, 6100, 771: 6098, 781: 2476, 785: 6097, 797: 6108, 832: 6104, 835: 6105, 846: 6103, 850: 6106, 6107, 907: 6113}, + {487: 6112}, + {464: 6113}, + {71: 2469, 145: 2471, 151: 2497, 153: 2468, 462: 2508, 2507, 491: 2506, 493: 6097, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6095, 730: 2475, 739: 6096, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6103, 6102, 2478, 761: 2604, 2476, 766: 6100, 6101, 769: 6099, 779: 2477, 783: 6098, 795: 6109, 830: 6105, 833: 6106, 844: 6104, 848: 6107, 6108, 905: 6114}, // 3655 {1993, 1993}, - {464: 6115}, - {71: 2468, 145: 2470, 151: 2496, 153: 2467, 462: 2507, 2506, 491: 2505, 495: 2491, 501: 6096, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6094, 732: 2474, 741: 6095, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6102, 6101, 2477, 763: 2603, 2475, 768: 6099, 6100, 771: 6098, 781: 2476, 785: 6097, 797: 6108, 832: 6104, 835: 6105, 846: 6103, 850: 6106, 6107, 907: 6116}, + {464: 6116}, + {71: 2469, 145: 2471, 151: 2497, 153: 2468, 462: 2508, 2507, 491: 2506, 493: 6097, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6095, 730: 2475, 739: 6096, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6103, 6102, 2478, 761: 2604, 2476, 766: 6100, 6101, 769: 6099, 779: 2477, 783: 6098, 795: 6109, 830: 6105, 833: 6106, 844: 6104, 848: 6107, 6108, 905: 6117}, {1995, 1995}, {2022, 2022}, // 3660 {2021, 2021}, {242, 242, 470: 242}, - {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 486: 1027, 738: 5197, 5196, 5195, 825: 5198, 873: 6121}, - {2: 1015, 1015, 1015, 1015, 1015, 8: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 51: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 6123, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 479: 1015, 486: 1015, 1089: 6122}, - {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4085, 486: 1813, 842: 6124}, + {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 486: 1027, 736: 5198, 5197, 5196, 823: 5199, 871: 6122}, + {2: 1015, 1015, 1015, 1015, 1015, 8: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 51: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 6124, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 479: 1015, 486: 1015, 1088: 6123}, + {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4086, 486: 1813, 840: 6125}, // 3665 {2: 1014, 1014, 1014, 1014, 1014, 8: 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 51: 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 1014, 479: 1014, 486: 1014}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 486: 6125, 653: 6127, 2674, 2675, 2673, 904: 6128, 951: 6126}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6142, 2674, 2675, 2673, 727: 6140, 904: 6128, 951: 6141}, - {7: 6136, 486: 6135}, - {7: 1017, 470: 1017, 486: 1017, 642: 6130, 894: 6129}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 486: 6126, 651: 6128, 2675, 2676, 2674, 902: 6129, 949: 6127}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6143, 2675, 2676, 2674, 725: 6141, 902: 6129, 949: 6142}, + {7: 6137, 486: 6136}, + {7: 1017, 470: 1017, 486: 1017, 640: 6131, 892: 6130}, // 3670 {7: 1019, 470: 1019, 486: 1019}, {7: 1021, 470: 1021, 486: 1021}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 6132, 653: 6131, 2674, 2675, 2673}, - {7: 1017, 470: 1017, 486: 1017, 642: 6134, 894: 6133}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 6133, 651: 6132, 2675, 2676, 2674}, + {7: 1017, 470: 1017, 486: 1017, 640: 6135, 892: 6134}, {7: 1016, 470: 1016, 486: 1016}, // 3675 {7: 1020, 470: 1020, 486: 1020}, - {512: 6132}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 568: 5204, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5211, 838: 5201, 876: 6138}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6127, 2674, 2675, 2673, 904: 6137}, + {512: 6133}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 568: 5205, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5212, 836: 5202, 874: 6139}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6128, 2675, 2676, 2674, 902: 6138}, {7: 1018, 470: 1018, 486: 1018}, // 3680 - {237, 237, 7: 5255, 470: 237, 489: 2634, 778: 2635, 6139}, + {237, 237, 7: 5256, 470: 237, 489: 2635, 776: 2636, 6140}, {2026, 2026, 470: 2026}, - {894, 894, 894, 894, 894, 894, 894, 8: 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 51: 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 467: 894, 470: 894, 479: 894, 5259, 483: 894, 489: 894, 894, 492: 894, 514: 894, 847: 6148}, - {7: 6136, 470: 6145}, - {1025, 1025, 1025, 1025, 1025, 1025, 1025, 1017, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 51: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 467: 1025, 470: 1017, 479: 1025, 1025, 483: 1025, 489: 1025, 1025, 492: 1025, 514: 1025, 642: 6143, 894: 6129}, + {894, 894, 894, 894, 894, 894, 894, 8: 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 51: 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 894, 467: 894, 470: 894, 479: 894, 5260, 483: 894, 489: 894, 894, 492: 894, 514: 894, 845: 6149}, + {7: 6137, 470: 6146}, + {1025, 1025, 1025, 1025, 1025, 1025, 1025, 1017, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 51: 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 1025, 467: 1025, 470: 1017, 479: 1025, 1025, 483: 1025, 489: 1025, 1025, 492: 1025, 514: 1025, 640: 6144, 892: 6130}, // 3685 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 512: 6132, 653: 6144, 2674, 2675, 2673}, - {1024, 1024, 1024, 1024, 1024, 1024, 1024, 1017, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 51: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 467: 1024, 470: 1017, 479: 1024, 1024, 483: 1024, 489: 1024, 1024, 492: 1024, 514: 1024, 642: 6134, 894: 6133}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 5209, 568: 5204, 653: 3818, 2674, 2675, 2673, 699: 5208, 727: 5207, 787: 5206, 790: 5205, 5211, 838: 5201, 876: 6146}, - {237, 237, 7: 5255, 489: 2634, 778: 2635, 6147}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 512: 6133, 651: 6145, 2675, 2676, 2674}, + {1024, 1024, 1024, 1024, 1024, 1024, 1024, 1017, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 51: 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 1024, 467: 1024, 470: 1017, 479: 1024, 1024, 483: 1024, 489: 1024, 1024, 492: 1024, 514: 1024, 640: 6135, 892: 6134}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 5210, 568: 5205, 651: 3819, 2675, 2676, 2674, 697: 5209, 725: 5208, 785: 5207, 788: 5206, 5212, 836: 5202, 874: 6147}, + {237, 237, 7: 5256, 489: 2635, 776: 2636, 6148}, {2025, 2025}, // 3690 - {892, 892, 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 467: 5216, 470: 892, 479: 892, 483: 892, 489: 892, 892, 492: 892, 514: 892, 653: 5215, 2674, 2675, 2673, 902: 5214, 6149}, - {873, 873, 470: 873, 479: 5269, 483: 873, 489: 873, 873, 492: 5270, 514: 5268, 927: 5272, 5271, 1043: 5273, 6150}, - {237, 237, 470: 237, 483: 237, 489: 2634, 237, 778: 2635, 6151}, - {1265, 1265, 470: 1265, 483: 1265, 490: 2637, 756: 2638, 799: 6152}, - {855, 855, 470: 855, 483: 5319, 1052: 6153}, + {892, 892, 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 467: 5217, 470: 892, 479: 892, 483: 892, 489: 892, 892, 492: 892, 514: 892, 651: 5216, 2675, 2676, 2674, 900: 5215, 6150}, + {873, 873, 470: 873, 479: 5270, 483: 873, 489: 873, 873, 492: 5271, 514: 5269, 925: 5273, 5272, 1041: 5274, 6151}, + {237, 237, 470: 237, 483: 237, 489: 2635, 237, 776: 2636, 6152}, + {1265, 1265, 470: 1265, 483: 1265, 490: 2638, 754: 2639, 797: 6153}, + {855, 855, 470: 855, 483: 5320, 1050: 6154}, // 3695 {2027, 2027, 470: 2027}, - {2028, 2028, 7: 3489}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6228, 2674, 2675, 2673}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6226}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6217}, + {2028, 2028, 7: 3490}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6229, 2675, 2676, 2674}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6227}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6218}, // 3700 - {110: 5333, 559: 5332, 1127: 6213}, - {154: 577, 160: 5387}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 555: 6208, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 6207}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 555: 6204, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 6203}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 555: 6200, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 6199}, + {110: 5334, 559: 5333, 1126: 6214}, + {154: 577, 160: 5388}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 555: 6209, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 6208}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 555: 6205, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 6204}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 555: 6201, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 6200}, // 3705 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6195}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6193}, - {154: 6173}, - {158: 6170}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6168}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6196}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6194}, + {154: 6174}, + {158: 6171}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6169}, // 3710 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 6169}, - {26, 26, 7: 3821}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6171}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4862, 2674, 2675, 2673, 872: 6172}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 6170}, + {26, 26, 7: 3822}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6172}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4863, 2675, 2676, 2674, 870: 6173}, {53, 53}, // 3715 - {482: 6174}, - {462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6177, 732: 6175, 741: 6178, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6180, 6179, 6176, 763: 2603, 6182, 768: 6183, 6184, 771: 6181, 880: 6185}, - {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 486: 825, 738: 825, 825, 825, 749: 5192, 853: 5193, 905: 6188}, - {462: 2507, 491: 2505, 559: 2504, 636: 2500, 644: 2604, 699: 3785, 741: 3784, 2501, 2502, 2503, 2512, 2510, 3786, 3787, 763: 6119}, - {175, 175, 469: 786, 175, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, + {482: 6175}, + {462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6178, 730: 6176, 739: 6179, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6181, 6180, 6177, 761: 2604, 6183, 766: 6184, 6185, 769: 6182, 878: 6186}, + {2: 825, 825, 825, 825, 825, 8: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 51: 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 825, 479: 825, 486: 825, 736: 825, 825, 825, 747: 5193, 851: 5194, 903: 6189}, + {462: 2508, 491: 2506, 559: 2505, 634: 2501, 642: 2605, 697: 3786, 739: 3785, 2502, 2503, 2504, 2513, 2511, 3787, 3788, 761: 6120}, + {175, 175, 469: 786, 175, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, // 3720 {177, 177, 469: 787, 177, 477: 787, 787}, {178, 178, 470: 178}, @@ -10491,144 +10486,144 @@ var ( // 3725 {172, 172, 470: 172}, {171, 171, 470: 171}, - {169, 169, 470: 6186}, - {462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6177, 732: 6175, 741: 6178, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6180, 6179, 6176, 763: 2603, 6182, 768: 6183, 6184, 771: 6181, 880: 6187}, + {169, 169, 470: 6187}, + {462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6178, 730: 6176, 739: 6179, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6181, 6180, 6177, 761: 2604, 6183, 766: 6184, 6185, 769: 6182, 878: 6188}, {168, 168}, // 3730 - {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 486: 1027, 738: 5197, 5196, 5195, 825: 5198, 873: 6189}, - {2: 1015, 1015, 1015, 1015, 1015, 8: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 51: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 6123, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 479: 1015, 486: 1015, 1089: 6190}, - {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4085, 486: 1813, 842: 6191}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 486: 6192, 653: 6127, 2674, 2675, 2673, 904: 6128, 951: 6126}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6140}, + {2: 1027, 1027, 1027, 1027, 1027, 8: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 51: 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 1027, 479: 1027, 486: 1027, 736: 5198, 5197, 5196, 823: 5199, 871: 6190}, + {2: 1015, 1015, 1015, 1015, 1015, 8: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 51: 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 6124, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 1015, 479: 1015, 486: 1015, 1088: 6191}, + {2: 1813, 1813, 1813, 1813, 1813, 8: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 51: 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 1813, 479: 4086, 486: 1813, 840: 6192}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 486: 6193, 651: 6128, 2675, 2676, 2674, 902: 6129, 949: 6127}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6141}, // 3735 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6194, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6195, 2675, 2676, 2674}, {1918, 1918}, - {2007, 2007, 156: 6197, 480: 6196}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 6198}, + {2007, 2007, 156: 6198, 480: 6197}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 6199}, {2005, 2005}, // 3740 - {2006, 2006, 7: 4627}, - {2009, 2009, 7: 5525}, - {573: 6201}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 5524, 5522, 849: 6202}, - {2008, 2008, 7: 5525}, + {2006, 2006, 7: 4628}, + {2009, 2009, 7: 5526}, + {573: 6202}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 5525, 5523, 847: 6203}, + {2008, 2008, 7: 5526}, // 3745 - {2011, 2011, 7: 5094}, - {573: 6205}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5092, 854: 6206}, - {2010, 2010, 7: 5094}, - {2004, 2004, 7: 3821, 660: 4678, 662: 4677, 898: 6212}, + {2011, 2011, 7: 5095}, + {573: 6206}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5093, 852: 6207}, + {2010, 2010, 7: 5095}, + {2004, 2004, 7: 3822, 658: 4679, 660: 4678, 896: 6213}, // 3750 - {573: 6209}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 6210}, - {2004, 2004, 7: 3821, 660: 4678, 662: 4677, 898: 6211}, + {573: 6210}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 6211}, + {2004, 2004, 7: 3822, 658: 4679, 660: 4678, 896: 6212}, {2012, 2012}, {2013, 2013}, // 3755 - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6214}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 3819, 784: 6215}, - {2004, 2004, 7: 3821, 660: 4678, 662: 4677, 898: 6216}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6215}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 3820, 782: 6216}, + {2004, 2004, 7: 3822, 658: 4679, 660: 4678, 896: 6217}, {2017, 2017}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6218, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6219, 2675, 2676, 2674}, // 3760 - {461: 6219}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6220}, - {2146, 2146, 81: 4126, 485: 4127, 856: 6222, 869: 6221, 1045: 6223}, - {2145, 2145, 81: 4126, 856: 6225}, - {2144, 2144, 485: 4127, 869: 6224}, + {461: 6220}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6221}, + {2147, 2147, 81: 4127, 485: 4128, 854: 6223, 867: 6222, 1043: 6224}, + {2146, 2146, 81: 4127, 854: 6226}, + {2145, 2145, 485: 4128, 867: 6225}, // 3765 {2018, 2018}, - {2142, 2142}, {2143, 2143}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 6227}, + {2144, 2144}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 6228}, {2019, 2019}, // 3770 - {2154, 2154}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6445}, - {639: 6433}, - {639: 2140}, - {639: 2139}, + {2155, 2155}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6451}, + {637: 6439}, + {637: 2141}, + {637: 2140}, // 3775 - {639: 2138}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6415}, - {16: 6336, 81: 6335, 99: 2045, 139: 2045, 658: 2045, 1309: 6334}, - {495: 6333}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6321}, + {637: 2139}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6416}, + {16: 6337, 81: 6336, 99: 2045, 139: 2045, 656: 2045, 1307: 6335}, + {496: 6334}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6322}, // 3780 - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 464: 1815, 555: 4702, 558: 1815, 773: 6289}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 464: 1815, 555: 4702, 773: 6283}, - {154: 6278}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6242}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6243}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 464: 1815, 555: 4703, 558: 1815, 771: 6290}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 464: 1815, 555: 4703, 771: 6284}, + {154: 6279}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6243}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6244}, // 3785 - {49, 49, 4: 49, 49, 49, 13: 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 70: 6251, 6248, 6254, 6255, 6256, 6249, 6247, 6257, 6253, 6250, 468: 49, 49, 473: 49, 492: 49, 494: 49, 637: 49, 639: 49, 647: 6252, 900: 6246, 1174: 6244, 1266: 6245}, - {385, 385, 4: 4132, 4134, 389, 13: 4151, 2118, 4149, 4090, 4153, 4140, 4133, 4136, 4169, 4135, 4138, 4139, 4141, 4148, 389, 4159, 4160, 4146, 4147, 4152, 4154, 4166, 4165, 4171, 4167, 4164, 4157, 4162, 4163, 4156, 4158, 4161, 4150, 468: 4131, 4168, 473: 2118, 492: 4845, 494: 2118, 637: 2118, 639: 4137, 766: 4142, 780: 4144, 800: 4143, 822: 4145, 826: 4155, 830: 4170, 906: 5409, 1001: 6277}, - {48, 48, 4: 48, 48, 48, 13: 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 70: 6251, 6248, 6254, 6255, 6256, 6249, 6247, 6257, 6253, 6250, 468: 48, 48, 473: 48, 492: 48, 494: 48, 637: 48, 639: 48, 647: 6252, 900: 6276}, - {47, 47, 4: 47, 47, 47, 13: 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 70: 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 468: 47, 47, 473: 47, 492: 47, 494: 47, 637: 47, 639: 47, 647: 47}, - {474: 1999, 1999, 487: 4172, 496: 1999, 649: 6273, 726: 6272}, + {49, 49, 4: 49, 49, 49, 13: 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 70: 6252, 6249, 6255, 6256, 6257, 6250, 6248, 6258, 6254, 6251, 468: 49, 49, 473: 49, 492: 49, 495: 49, 635: 49, 637: 49, 645: 6253, 898: 6247, 1173: 6245, 1264: 6246}, + {385, 385, 4: 4133, 4135, 389, 13: 4152, 2118, 4150, 4091, 4154, 4141, 4170, 4134, 4137, 4136, 4139, 4140, 4142, 4149, 389, 4160, 4161, 4147, 4148, 4153, 4155, 4167, 4166, 4172, 4168, 4165, 4158, 4163, 4164, 4157, 4159, 4162, 4151, 468: 4132, 4169, 473: 2118, 492: 4846, 495: 2118, 635: 2118, 637: 4138, 764: 4143, 778: 4145, 798: 4144, 820: 4146, 824: 4156, 828: 4171, 904: 5410, 999: 6278}, + {48, 48, 4: 48, 48, 48, 13: 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 70: 6252, 6249, 6255, 6256, 6257, 6250, 6248, 6258, 6254, 6251, 468: 48, 48, 473: 48, 492: 48, 495: 48, 635: 48, 637: 48, 645: 6253, 898: 6277}, + {47, 47, 4: 47, 47, 47, 13: 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 70: 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 468: 47, 47, 473: 47, 492: 47, 495: 47, 635: 47, 637: 47, 645: 47}, + {474: 1999, 1999, 487: 4173, 497: 1999, 647: 6274, 724: 6273}, // 3790 - {463: 6269, 474: 1999, 1999, 487: 4172, 496: 1999, 726: 6268}, - {474: 1999, 1999, 487: 4172, 496: 1999, 726: 6266}, - {40, 40, 4: 40, 40, 40, 13: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 70: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 468: 40, 40, 473: 40, 492: 40, 494: 40, 637: 40, 639: 40, 647: 40}, - {72: 6264, 74: 6265, 6262, 647: 6263}, - {474: 1999, 1999, 487: 4172, 496: 1999, 726: 6260}, + {463: 6270, 474: 1999, 1999, 487: 4173, 497: 1999, 724: 6269}, + {474: 1999, 1999, 487: 4173, 497: 1999, 724: 6267}, + {40, 40, 4: 40, 40, 40, 13: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 70: 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 468: 40, 40, 473: 40, 492: 40, 495: 40, 635: 40, 637: 40, 645: 40}, + {72: 6265, 74: 6266, 6263, 645: 6264}, + {474: 1999, 1999, 487: 4173, 497: 1999, 724: 6261}, // 3795 - {37, 37, 4: 37, 37, 37, 13: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 70: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 468: 37, 37, 473: 37, 492: 37, 494: 37, 637: 37, 639: 37, 647: 37}, - {474: 1999, 1999, 487: 4172, 496: 1999, 726: 6258}, - {34, 34, 4: 34, 34, 34, 13: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 70: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 468: 34, 34, 473: 34, 492: 34, 494: 34, 637: 34, 639: 34, 647: 34}, - {32, 32, 4: 32, 32, 32, 13: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 70: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 468: 32, 32, 473: 32, 492: 32, 494: 32, 637: 32, 639: 32, 647: 32}, - {31, 31, 4: 31, 31, 31, 13: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 70: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 468: 31, 31, 473: 31, 492: 31, 494: 31, 637: 31, 639: 31, 647: 31}, + {37, 37, 4: 37, 37, 37, 13: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 70: 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 468: 37, 37, 473: 37, 492: 37, 495: 37, 635: 37, 637: 37, 645: 37}, + {474: 1999, 1999, 487: 4173, 497: 1999, 724: 6259}, + {34, 34, 4: 34, 34, 34, 13: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 70: 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 468: 34, 34, 473: 34, 492: 34, 495: 34, 635: 34, 637: 34, 645: 34}, + {32, 32, 4: 32, 32, 32, 13: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 70: 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 468: 32, 32, 473: 32, 492: 32, 495: 32, 635: 32, 637: 32, 645: 32}, + {31, 31, 4: 31, 31, 31, 13: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 70: 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 468: 31, 31, 473: 31, 492: 31, 495: 31, 635: 31, 637: 31, 645: 31}, // 3800 - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6259}, - {35, 35, 4: 35, 35, 35, 13: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 70: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 468: 35, 35, 473: 35, 492: 35, 494: 35, 637: 35, 639: 35, 647: 35}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6261}, - {38, 38, 4: 38, 38, 38, 13: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 70: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 468: 38, 38, 473: 38, 492: 38, 494: 38, 637: 38, 639: 38, 647: 38}, - {39, 39, 4: 39, 39, 39, 13: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 70: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 468: 39, 39, 473: 39, 492: 39, 494: 39, 637: 39, 639: 39, 647: 39}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6260}, + {35, 35, 4: 35, 35, 35, 13: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 70: 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 468: 35, 35, 473: 35, 492: 35, 495: 35, 635: 35, 637: 35, 645: 35}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6262}, + {38, 38, 4: 38, 38, 38, 13: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 70: 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 468: 38, 38, 473: 38, 492: 38, 495: 38, 635: 38, 637: 38, 645: 38}, + {39, 39, 4: 39, 39, 39, 13: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 70: 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 468: 39, 39, 473: 39, 492: 39, 495: 39, 635: 39, 637: 39, 645: 39}, // 3805 - {36, 36, 4: 36, 36, 36, 13: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 70: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 468: 36, 36, 473: 36, 492: 36, 494: 36, 637: 36, 639: 36, 647: 36}, - {33, 33, 4: 33, 33, 33, 13: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 70: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 468: 33, 33, 473: 33, 492: 33, 494: 33, 637: 33, 639: 33, 647: 33}, - {30, 30, 4: 30, 30, 30, 13: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 70: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 468: 30, 30, 473: 30, 492: 30, 494: 30, 637: 30, 639: 30, 647: 30}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6267}, - {41, 41, 4: 41, 41, 41, 13: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 70: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 468: 41, 41, 473: 41, 492: 41, 494: 41, 637: 41, 639: 41, 647: 41}, + {36, 36, 4: 36, 36, 36, 13: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 70: 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 468: 36, 36, 473: 36, 492: 36, 495: 36, 635: 36, 637: 36, 645: 36}, + {33, 33, 4: 33, 33, 33, 13: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 70: 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 468: 33, 33, 473: 33, 492: 33, 495: 33, 635: 33, 637: 33, 645: 33}, + {30, 30, 4: 30, 30, 30, 13: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 70: 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 468: 30, 30, 473: 30, 492: 30, 495: 30, 635: 30, 637: 30, 645: 30}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6268}, + {41, 41, 4: 41, 41, 41, 13: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 70: 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 468: 41, 41, 473: 41, 492: 41, 495: 41, 635: 41, 637: 41, 645: 41}, // 3810 - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6271}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6270}, - {42, 42, 4: 42, 42, 42, 13: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 70: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 468: 42, 42, 473: 42, 492: 42, 494: 42, 637: 42, 639: 42, 647: 42}, - {43, 43, 4: 43, 43, 43, 13: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 70: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 468: 43, 43, 473: 43, 492: 43, 494: 43, 637: 43, 639: 43, 647: 43}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6275}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6272}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6271}, + {42, 42, 4: 42, 42, 42, 13: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 70: 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 468: 42, 42, 473: 42, 492: 42, 495: 42, 635: 42, 637: 42, 645: 42}, + {43, 43, 4: 43, 43, 43, 13: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 70: 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 468: 43, 43, 473: 43, 492: 43, 495: 43, 635: 43, 637: 43, 645: 43}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6276}, // 3815 - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6274}, - {44, 44, 4: 44, 44, 44, 13: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 70: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 468: 44, 44, 473: 44, 492: 44, 494: 44, 637: 44, 639: 44, 647: 44}, - {45, 45, 4: 45, 45, 45, 13: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 70: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 468: 45, 45, 473: 45, 492: 45, 494: 45, 637: 45, 639: 45, 647: 45}, - {46, 46, 4: 46, 46, 46, 13: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 70: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 468: 46, 46, 473: 46, 492: 46, 494: 46, 637: 46, 639: 46, 647: 46}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6275}, + {44, 44, 4: 44, 44, 44, 13: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 70: 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 468: 44, 44, 473: 44, 492: 44, 495: 44, 635: 44, 637: 44, 645: 44}, + {45, 45, 4: 45, 45, 45, 13: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 70: 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 468: 45, 45, 473: 45, 492: 45, 495: 45, 635: 45, 637: 45, 645: 45}, + {46, 46, 4: 46, 46, 46, 13: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 70: 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 468: 46, 46, 473: 46, 492: 46, 495: 46, 635: 46, 637: 46, 645: 46}, {50, 50}, // 3820 - {482: 6279}, - {462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6177, 732: 6175, 741: 6178, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6180, 6179, 6176, 763: 2603, 6182, 768: 6183, 6184, 771: 6181, 880: 6280}, - {470: 6281}, - {462: 2507, 2506, 491: 2505, 495: 2491, 556: 2490, 559: 2504, 636: 2500, 644: 2604, 699: 6177, 732: 6175, 741: 6178, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6180, 6179, 6176, 763: 2603, 6182, 768: 6183, 6184, 771: 6181, 880: 6282}, + {482: 6280}, + {462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6178, 730: 6176, 739: 6179, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6181, 6180, 6177, 761: 2604, 6183, 766: 6184, 6185, 769: 6182, 878: 6281}, + {470: 6282}, + {462: 2508, 2507, 491: 2506, 496: 2492, 556: 2491, 559: 2505, 634: 2501, 642: 2605, 697: 6178, 730: 6176, 739: 6179, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6181, 6180, 6177, 761: 2604, 6183, 766: 6184, 6185, 769: 6182, 878: 6283}, {170, 170}, // 3825 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 6285, 5522, 1102: 6286, 1261: 6284}, - {232, 232, 7: 6287}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 6286, 5523, 1101: 6287, 1259: 6285}, + {232, 232, 7: 6288}, {181, 181, 7: 181}, {180, 180, 7: 180}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 5516, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 5521, 653: 3390, 2674, 2675, 2673, 728: 5045, 789: 5523, 808: 6285, 5522, 1102: 6288}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 5517, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 5522, 651: 3391, 2675, 2676, 2674, 726: 5046, 787: 5524, 806: 6286, 5523, 1101: 6289}, // 3830 {179, 179, 7: 179}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5108, 877: 5109, 909: 6290}, - {215, 215, 7: 5111, 13: 215, 46: 215, 463: 215, 651: 5155, 944: 5154, 6291}, - {223, 223, 13: 223, 46: 223, 463: 6293, 992: 6292}, - {202, 202, 13: 6310, 46: 6308, 937: 6309, 6307, 1082: 6306, 6305}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5109, 875: 5110, 907: 6291}, + {215, 215, 7: 5112, 13: 215, 46: 215, 463: 215, 649: 5156, 942: 5155, 6292}, + {223, 223, 13: 223, 46: 223, 463: 6294, 990: 6293}, + {202, 202, 13: 6311, 46: 6309, 935: 6310, 6308, 1081: 6307, 6306}, // 3835 - {126: 6298, 6296, 6297, 6299, 991: 6295, 1172: 6294}, - {222, 222, 13: 222, 46: 222, 126: 6298, 6296, 6297, 6299, 991: 6304}, + {126: 6299, 6297, 6298, 6300, 989: 6296, 1171: 6295}, + {222, 222, 13: 222, 46: 222, 126: 6299, 6297, 6298, 6300, 989: 6305}, {221, 221, 13: 221, 46: 221, 126: 221, 221, 221, 221}, - {496: 2648, 725: 3948, 751: 6303}, - {496: 2648, 725: 3948, 751: 6302}, + {497: 2649, 721: 3949, 749: 6304}, + {497: 2649, 721: 3949, 749: 6303}, // 3840 - {496: 2648, 725: 3948, 751: 6301}, - {496: 2648, 725: 3948, 751: 6300}, + {497: 2649, 721: 3949, 749: 6302}, + {497: 2649, 721: 3949, 749: 6301}, {216, 216, 13: 216, 46: 216, 126: 216, 216, 216, 216}, {217, 217, 13: 217, 46: 217, 126: 217, 217, 217, 217}, {218, 218, 13: 218, 46: 218, 126: 218, 218, 218, 218}, @@ -10636,377 +10631,383 @@ var ( {219, 219, 13: 219, 46: 219, 126: 219, 219, 219, 219}, {220, 220, 13: 220, 46: 220, 126: 220, 220, 220, 220}, {233, 233}, - {201, 201, 13: 6310, 46: 6308, 937: 6309, 6320}, + {201, 201, 13: 6311, 46: 6309, 935: 6310, 6321}, {200, 200, 13: 200, 46: 200}, // 3850 - {485: 6319, 957: 6318}, - {196, 196, 13: 196, 46: 196, 204: 6314, 468: 6315, 570: 6313}, - {322: 6311}, - {191, 191, 13: 191, 46: 191, 204: 191, 468: 191, 570: 191, 1164: 6312}, + {485: 6320, 955: 6319}, + {196, 196, 13: 196, 46: 196, 204: 6315, 468: 6316, 570: 6314}, + {324: 6312}, + {191, 191, 13: 191, 46: 191, 204: 191, 468: 191, 570: 191, 1163: 6313}, {192, 192, 13: 192, 46: 192, 204: 192, 468: 192, 570: 192}, // 3855 - {496: 2648, 725: 3948, 751: 6316}, + {497: 2649, 721: 3949, 749: 6317}, {194, 194, 13: 194, 46: 194}, {193, 193, 13: 193, 46: 193}, - {106: 6317}, + {106: 6318}, {195, 195, 13: 195, 46: 195}, // 3860 {198, 198, 13: 198, 46: 198}, {197, 197, 13: 197, 46: 197}, {199, 199, 13: 199, 46: 199}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6322, 2674, 2675, 2673}, - {486: 6323}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6323, 2675, 2676, 2674}, + {486: 6324}, // 3865 - {464: 6324}, - {1910, 1910, 15: 1910, 47: 1910, 49: 1910, 51: 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 137: 6327, 461: 1910, 495: 6326, 643: 1910, 1025: 6325}, - {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 643: 1967, 885: 6332}, - {1909, 1909, 15: 1909, 47: 1909, 49: 1909, 51: 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 461: 1909, 643: 1909}, - {194: 6330, 376: 6331, 633: 6329, 641: 6328}, + {464: 6325}, + {1910, 1910, 15: 1910, 47: 1910, 49: 1910, 51: 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 137: 6328, 461: 1910, 496: 6327, 641: 1910, 1023: 6326}, + {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 641: 1967, 883: 6333}, + {1909, 1909, 15: 1909, 47: 1909, 49: 1909, 51: 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 1909, 461: 1909, 641: 1909}, + {195: 6331, 377: 6332, 631: 6330, 639: 6329}, // 3870 - {1908, 1908, 15: 1908, 47: 1908, 49: 1908, 51: 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 461: 1908, 643: 1908}, - {1907, 1907, 15: 1907, 47: 1907, 49: 1907, 51: 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 461: 1907, 643: 1907}, - {1906, 1906, 15: 1906, 47: 1906, 49: 1906, 51: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 461: 1906, 643: 1906}, - {1905, 1905, 15: 1905, 47: 1905, 49: 1905, 51: 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 461: 1905, 643: 1905}, - {1922, 1922, 15: 6019, 47: 5995, 49: 6015, 51: 6008, 5998, 5994, 6002, 6006, 6018, 6001, 6007, 6005, 6003, 6016, 6009, 5997, 6017, 5996, 5999, 6000, 6004, 461: 6010, 643: 6020, 881: 6012, 6011, 6014, 5993, 886: 6013}, + {1908, 1908, 15: 1908, 47: 1908, 49: 1908, 51: 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 1908, 461: 1908, 641: 1908}, + {1907, 1907, 15: 1907, 47: 1907, 49: 1907, 51: 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 1907, 461: 1907, 641: 1907}, + {1906, 1906, 15: 1906, 47: 1906, 49: 1906, 51: 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 1906, 461: 1906, 641: 1906}, + {1905, 1905, 15: 1905, 47: 1905, 49: 1905, 51: 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 1905, 461: 1905, 641: 1905}, + {1922, 1922, 15: 6020, 47: 5996, 49: 6016, 51: 6009, 5999, 5995, 6003, 6007, 6019, 6002, 6008, 6006, 6004, 6017, 6010, 5998, 6018, 5997, 6000, 6001, 6005, 461: 6011, 641: 6021, 879: 6013, 6012, 6015, 5994, 884: 6014}, // 3875 - {16: 2046, 81: 2046, 99: 2046, 139: 2046, 658: 2046}, - {99: 2041, 139: 6383, 658: 2041, 1311: 6382}, - {487: 6378}, - {158: 6337}, - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6338}, + {16: 2046, 81: 2046, 99: 2046, 139: 2046, 656: 2046}, + {99: 2041, 139: 6384, 656: 2041, 1309: 6383}, + {487: 6379}, + {158: 6338}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6339}, // 3880 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4862, 2674, 2675, 2673, 872: 6339}, - {86: 6343, 88: 6348, 6350, 6344, 6349, 6352, 6346, 6342, 6347, 6351, 6345, 860: 6340, 1084: 6341}, - {2453, 2453, 7: 2453, 86: 2453, 88: 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453}, - {52, 52, 7: 6376, 86: 6343, 88: 6348, 6350, 6344, 6349, 6352, 6346, 6342, 6347, 6351, 6345, 860: 6375}, - {464: 1999, 487: 4172, 726: 6373}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4863, 2675, 2676, 2674, 870: 6340}, + {86: 6344, 88: 6349, 6351, 6345, 6350, 6353, 6347, 6343, 6348, 6352, 6346, 858: 6341, 1083: 6342}, + {2454, 2454, 7: 2454, 86: 2454, 88: 2454, 2454, 2454, 2454, 2454, 2454, 2454, 2454, 2454, 2454}, + {52, 52, 7: 6377, 86: 6344, 88: 6349, 6351, 6345, 6350, 6353, 6347, 6343, 6348, 6352, 6346, 858: 6376}, + {464: 1999, 487: 4173, 724: 6374}, // 3885 - {464: 1999, 487: 4172, 726: 6371}, - {487: 4172, 496: 1999, 726: 6369}, - {487: 4172, 496: 1999, 726: 6367}, - {487: 4172, 496: 1999, 726: 6365}, - {464: 1999, 487: 4172, 726: 6363}, + {464: 1999, 487: 4173, 724: 6372}, + {487: 4173, 497: 1999, 724: 6370}, + {487: 4173, 497: 1999, 724: 6368}, + {487: 4173, 497: 1999, 724: 6366}, + {464: 1999, 487: 4173, 724: 6364}, // 3890 - {464: 1999, 487: 4172, 726: 6361}, - {464: 1999, 487: 4172, 726: 6359}, - {464: 1999, 487: 4172, 726: 6357}, - {464: 1999, 487: 4172, 726: 6355}, - {464: 1999, 487: 4172, 726: 6353}, + {464: 1999, 487: 4173, 724: 6362}, + {464: 1999, 487: 4173, 724: 6360}, + {464: 1999, 487: 4173, 724: 6358}, + {464: 1999, 487: 4173, 724: 6356}, + {464: 1999, 487: 4173, 724: 6354}, // 3895 - {464: 6354}, - {2440, 2440, 7: 2440, 86: 2440, 88: 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440, 2440}, - {464: 6356}, + {464: 6355}, {2441, 2441, 7: 2441, 86: 2441, 88: 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441, 2441}, - {464: 6358}, - // 3900 + {464: 6357}, {2442, 2442, 7: 2442, 86: 2442, 88: 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442, 2442}, - {464: 6360}, + {464: 6359}, + // 3900 {2443, 2443, 7: 2443, 86: 2443, 88: 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443, 2443}, - {464: 6362}, + {464: 6361}, {2444, 2444, 7: 2444, 86: 2444, 88: 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444, 2444}, - // 3905 - {464: 6364}, + {464: 6363}, {2445, 2445, 7: 2445, 86: 2445, 88: 2445, 2445, 2445, 2445, 2445, 2445, 2445, 2445, 2445, 2445}, - {496: 2648, 725: 2647, 734: 6366}, + // 3905 + {464: 6365}, {2446, 2446, 7: 2446, 86: 2446, 88: 2446, 2446, 2446, 2446, 2446, 2446, 2446, 2446, 2446, 2446}, - {496: 2648, 725: 2647, 734: 6368}, - // 3910 + {497: 2649, 721: 2648, 731: 6367}, {2447, 2447, 7: 2447, 86: 2447, 88: 2447, 2447, 2447, 2447, 2447, 2447, 2447, 2447, 2447, 2447}, - {496: 2648, 725: 2647, 734: 6370}, + {497: 2649, 721: 2648, 731: 6369}, + // 3910 {2448, 2448, 7: 2448, 86: 2448, 88: 2448, 2448, 2448, 2448, 2448, 2448, 2448, 2448, 2448, 2448}, - {464: 6372}, + {497: 2649, 721: 2648, 731: 6371}, {2449, 2449, 7: 2449, 86: 2449, 88: 2449, 2449, 2449, 2449, 2449, 2449, 2449, 2449, 2449, 2449}, - // 3915 - {464: 6374}, + {464: 6373}, {2450, 2450, 7: 2450, 86: 2450, 88: 2450, 2450, 2450, 2450, 2450, 2450, 2450, 2450, 2450, 2450}, - {2452, 2452, 7: 2452, 86: 2452, 88: 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452}, - {86: 6343, 88: 6348, 6350, 6344, 6349, 6352, 6346, 6342, 6347, 6351, 6345, 860: 6377}, + // 3915 + {464: 6375}, {2451, 2451, 7: 2451, 86: 2451, 88: 2451, 2451, 2451, 2451, 2451, 2451, 2451, 2451, 2451, 2451}, + {2453, 2453, 7: 2453, 86: 2453, 88: 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453, 2453}, + {86: 6344, 88: 6349, 6351, 6345, 6350, 6353, 6347, 6343, 6348, 6352, 6346, 858: 6378}, + {2452, 2452, 7: 2452, 86: 2452, 88: 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452, 2452}, // 3920 - {344: 6380, 382: 6381, 392: 6379}, - {99: 2044, 139: 2044, 658: 2044}, - {99: 2043, 139: 2043, 658: 2043}, - {99: 2042, 139: 2042, 658: 2042}, - {99: 2039, 658: 6387, 1314: 6386}, + {346: 6381, 383: 6382, 392: 6380}, + {99: 2044, 139: 2044, 656: 2044}, + {99: 2043, 139: 2043, 656: 2043}, + {99: 2042, 139: 2042, 656: 2042}, + {99: 2039, 656: 6388, 1312: 6387}, // 3925 - {487: 6384}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 6385}, - {99: 2040, 658: 2040}, - {99: 6391}, - {369: 6388}, + {487: 6385}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 6386}, + {99: 2040, 656: 2040}, + {99: 6392}, + {370: 6389}, // 3930 - {139: 6389, 335: 6390}, + {139: 6390, 337: 6391}, {99: 2038}, {99: 2037}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6393, 1313: 6392}, - {462: 6395, 467: 2035, 1312: 6394}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6394, 1311: 6393}, + {462: 6396, 467: 2035, 1310: 6395}, // 3935 {462: 2036, 467: 2036}, - {467: 6401}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6397, 2674, 2675, 2673, 1166: 6396}, - {7: 6399, 50: 6398}, + {467: 6402}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6398, 2675, 2676, 2674, 1165: 6397}, + {7: 6400, 50: 6399}, {7: 2033, 50: 2033}, // 3940 {467: 2034}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6400, 2674, 2675, 2673}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6401, 2675, 2676, 2674}, {7: 2032, 50: 2032}, - {462: 2507, 2506, 491: 2505, 559: 2504, 636: 2500, 699: 6405, 741: 6403, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 6404, 6402, 3777, 1176: 6406}, + {462: 2508, 2507, 491: 2506, 559: 2505, 634: 2501, 697: 6406, 739: 6404, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 6405, 6403, 3778, 1175: 6407}, {2054, 2054, 463: 2054}, // 3945 {2053, 2053, 463: 2053, 469: 787, 477: 787, 787}, {2052, 2052, 463: 2052}, - {2051, 2051, 463: 2051, 469: 786, 477: 786, 786, 483: 2640, 488: 2641, 490: 2637, 756: 3788, 3789}, - {2031, 2031, 463: 6408, 1310: 6407}, + {2051, 2051, 463: 2051, 469: 786, 477: 786, 786, 483: 2641, 488: 2642, 490: 2638, 754: 3789, 3790}, + {2031, 2031, 463: 6409, 1308: 6408}, {2048, 2048}, // 3950 - {136: 6410, 304: 6409}, - {574: 6413}, - {574: 6411}, - {893: 6412}, + {136: 6411, 306: 6410}, + {574: 6414}, + {574: 6412}, + {891: 6413}, {2029, 2029}, // 3955 - {893: 6414}, + {891: 6415}, {2030, 2030}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5190, 2674, 2675, 2673, 803: 6416}, - {2127, 2127, 14: 2118, 16: 2118, 21: 2118, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 766: 6419, 780: 6418, 836: 6421, 919: 6420, 1177: 6417}, - {2135, 2135}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5191, 2675, 2676, 2674, 801: 6417}, + {2127, 2127, 14: 2118, 16: 2118, 19: 2118, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 764: 6420, 778: 6419, 834: 6423, 917: 6422, 1176: 6418}, + {2136, 2136}, // 3960 - {14: 3733, 16: 4090, 21: 6425, 473: 6424, 494: 3734, 637: 3732, 761: 6423, 766: 6426}, - {2128, 2128, 14: 2128, 16: 2128, 21: 2128, 468: 2128, 473: 2128, 494: 2128, 637: 2128}, - {2126, 2126, 14: 2118, 16: 2118, 21: 2118, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 766: 6419, 780: 6418, 836: 6422}, - {2125, 2125, 14: 2125, 16: 2125, 21: 2125, 468: 2125, 473: 2125, 494: 2125, 637: 2125}, - {2124, 2124, 14: 2124, 16: 2124, 21: 2124, 468: 2124, 473: 2124, 494: 2124, 637: 2124}, + {14: 3734, 16: 4091, 19: 6431, 473: 6430, 495: 3735, 635: 3733, 759: 6429, 764: 6432}, + {2129, 2129, 14: 2129, 16: 2129, 19: 2129, 468: 2129, 473: 2129, 493: 2129, 495: 2129, 635: 2129}, + {282: 6425}, + {2126, 2126, 14: 2118, 16: 2118, 19: 2118, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 764: 6420, 778: 6419, 834: 6424}, + {2125, 2125, 14: 2125, 16: 2125, 19: 2125, 468: 2125, 473: 2125, 493: 2125, 495: 2125, 635: 2125}, // 3965 - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 532: 1999, 726: 6431}, - {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4172, 532: 1999, 726: 6429}, - {464: 1999, 487: 4172, 726: 6427}, - {2129, 2129, 14: 2129, 16: 2129, 21: 2129, 468: 2129, 473: 2129, 494: 2129, 637: 2129}, - {464: 4190, 1023: 6428}, + {2124, 2124, 14: 2124, 16: 2124, 19: 2124, 468: 2124, 473: 2124, 493: 2124, 495: 2124, 635: 2124}, + {267: 6426}, + {497: 2649, 721: 2648, 731: 6427}, + {2427, 2427, 14: 2427, 16: 2427, 19: 2427, 176: 4839, 468: 2427, 473: 2427, 493: 2427, 495: 2427, 635: 2427, 1056: 6428}, + {2128, 2128, 14: 2128, 16: 2128, 19: 2128, 468: 2128, 473: 2128, 493: 2128, 495: 2128, 635: 2128}, // 3970 - {2130, 2130, 14: 2130, 16: 2130, 21: 2130, 468: 2130, 473: 2130, 494: 2130, 637: 2130}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3388, 653: 3390, 2674, 2675, 2673, 728: 3387, 858: 6430}, - {2131, 2131, 14: 2131, 16: 2131, 21: 2131, 468: 2131, 473: 2131, 494: 2131, 637: 2131}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 532: 3658, 653: 3390, 2674, 2675, 2673, 728: 3657, 795: 6432}, - {2132, 2132, 14: 2132, 16: 2132, 21: 2132, 468: 2132, 473: 2132, 494: 2132, 637: 2132}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 534: 1999, 724: 6437}, + {2: 1999, 1999, 1999, 1999, 1999, 8: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 51: 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 1999, 464: 1999, 487: 4173, 534: 1999, 724: 6435}, + {464: 1999, 487: 4173, 724: 6433}, + {2130, 2130, 14: 2130, 16: 2130, 19: 2130, 468: 2130, 473: 2130, 493: 2130, 495: 2130, 635: 2130}, + {464: 4191, 1021: 6434}, // 3975 - {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4702, 773: 6434}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6435, 2674, 2675, 2673}, - {82: 4735, 461: 1798, 470: 4734, 845: 6437, 1210: 6436}, - {461: 6438}, - {461: 1797}, + {2131, 2131, 14: 2131, 16: 2131, 19: 2131, 468: 2131, 473: 2131, 493: 2131, 495: 2131, 635: 2131}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3389, 651: 3391, 2675, 2676, 2674, 726: 3388, 856: 6436}, + {2132, 2132, 14: 2132, 16: 2132, 19: 2132, 468: 2132, 473: 2132, 493: 2132, 495: 2132, 635: 2132}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 534: 3659, 651: 3391, 2675, 2676, 2674, 726: 3658, 793: 6438}, + {2133, 2133, 14: 2133, 16: 2133, 19: 2133, 468: 2133, 473: 2133, 493: 2133, 495: 2133, 635: 2133}, // 3980 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6439}, - {462: 6440}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 462: 4508, 653: 4038, 2674, 2675, 2673, 733: 4507, 814: 4506, 823: 6441}, - {7: 4517, 50: 6442}, - {1809, 1809, 4: 1809, 17: 1809, 81: 1809, 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 485: 1809, 866: 6443}, + {2: 1815, 1815, 1815, 1815, 1815, 8: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 51: 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 1815, 555: 4703, 771: 6440}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6441, 2675, 2676, 2674}, + {82: 4736, 461: 1798, 470: 4735, 843: 6443, 1209: 6442}, + {461: 6444}, + {461: 1797}, // 3985 - {2146, 2146, 4: 4731, 17: 4728, 81: 4126, 4735, 4595, 4320, 4596, 87: 4319, 463: 4730, 470: 4734, 485: 4127, 843: 4732, 845: 4729, 855: 4733, 6222, 865: 4727, 869: 6221, 1045: 6444}, - {2153, 2153}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6446, 2674, 2675, 2673}, - {462: 6447}, - {222: 4764, 231: 4766, 234: 4765, 1119: 6448}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6445}, + {462: 6446}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 462: 4509, 651: 4039, 2675, 2676, 2674, 732: 4508, 812: 4507, 821: 6447}, + {7: 4518, 50: 6448}, + {1809, 1809, 4: 1809, 17: 1809, 81: 1809, 1809, 1809, 1809, 1809, 87: 1809, 463: 1809, 470: 1809, 485: 1809, 864: 6449}, // 3990 - {50: 6449}, - {461: 6450}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6451}, - {462: 6452}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4038, 2674, 2675, 2673, 733: 4039, 796: 6453}, + {2147, 2147, 4: 4732, 17: 4729, 81: 4127, 4736, 4596, 4321, 4597, 87: 4320, 463: 4731, 470: 4735, 485: 4128, 841: 4733, 843: 4730, 853: 4734, 6223, 863: 4728, 867: 6222, 1043: 6450}, + {2154, 2154}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6452, 2675, 2676, 2674}, + {462: 6453}, + {222: 4765, 231: 4767, 234: 4766, 1118: 6454}, // 3995 - {7: 4041, 50: 6454}, - {2155, 2155}, - {2247, 2247}, - {2272, 2272}, - {2278, 2278, 463: 6459, 661: 6458}, + {50: 6455}, + {461: 6456}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6457}, + {462: 6458}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4039, 2675, 2676, 2674, 732: 4040, 794: 6459}, // 4000 - {150: 6466, 677: 6465}, - {305: 6461, 313: 6460}, - {49: 6464}, - {312: 6462}, - {150: 6463}, + {7: 4042, 50: 6460}, + {2156, 2156}, + {2248, 2248}, + {2273, 2273}, + {2279, 2279, 463: 6465, 659: 6464}, // 4005 - {2275, 2275}, + {150: 6472, 675: 6471}, + {307: 6467, 315: 6466}, + {49: 6470}, + {314: 6468}, + {150: 6469}, + // 4010 {2276, 2276}, {2277, 2277}, - {2274, 2274, 663: 5265, 911: 6467}, - {2273, 2273}, - // 4010 - {2280, 2280}, - {2279, 2279}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6483, 784: 6482}, - {559: 6472}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6473}, + {2278, 2278}, + {2275, 2275, 661: 5266, 909: 6473}, + {2274, 2274}, // 4015 - {480: 6475, 639: 6474}, - {880, 880, 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 880, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 6480}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 6476}, - {7: 4627, 639: 6477}, - {880, 880, 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 880, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 6478}, + {2281, 2281}, + {2280, 2280}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6489, 782: 6488}, + {559: 6478}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6479}, // 4020 - {2296, 2296, 7: 4886, 463: 4867, 792: 6479}, - {2304, 2304}, - {2296, 2296, 7: 4886, 463: 4867, 792: 6481}, - {2307, 2307}, - {2299, 2299, 7: 3821, 159: 6503, 463: 2299, 641: 6502, 965: 6513}, + {480: 6481, 637: 6480}, + {880, 880, 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 880, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 6486}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 6482}, + {7: 4628, 637: 6483}, + {880, 880, 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 880, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 6484}, // 4025 - {1023, 1023, 7: 1023, 98: 6488, 159: 1023, 463: 1023, 480: 6485, 639: 6484, 641: 1023, 644: 6486, 659: 6487}, - {880, 880, 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 880, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 6511}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4626, 2674, 2675, 2673, 777: 6498}, - {246: 6494}, - {246: 6491}, + {2297, 2297, 7: 4887, 463: 4868, 790: 6485}, + {2305, 2305}, + {2297, 2297, 7: 4887, 463: 4868, 790: 6487}, + {2308, 2308}, + {2300, 2300, 7: 3822, 159: 6509, 463: 2300, 639: 6508, 963: 6519}, // 4030 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5640, 2674, 2675, 2673, 863: 6489}, - {2296, 2296, 7: 5642, 463: 4867, 792: 6490}, - {2301, 2301}, - {461: 6492}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5640, 2674, 2675, 2673, 863: 6493}, + {1023, 1023, 7: 1023, 98: 6494, 159: 1023, 463: 1023, 480: 6491, 637: 6490, 639: 1023, 642: 6492, 657: 6493}, + {880, 880, 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 880, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 6517}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4627, 2675, 2676, 2674, 775: 6504}, + {246: 6500}, + {246: 6497}, // 4035 - {2302, 2302, 7: 5642}, - {461: 6495}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5640, 2674, 2675, 2673, 863: 6496}, - {2296, 2296, 7: 5642, 463: 4867, 792: 6497}, - {2303, 2303}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5641, 2675, 2676, 2674, 861: 6495}, + {2297, 2297, 7: 5643, 463: 4868, 790: 6496}, + {2302, 2302}, + {461: 6498}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5641, 2675, 2676, 2674, 861: 6499}, // 4040 - {2299, 2299, 7: 4627, 98: 6501, 159: 6503, 463: 2299, 639: 6500, 641: 6502, 965: 6499}, - {2296, 2296, 463: 4867, 792: 6510}, - {880, 880, 2910, 2758, 2794, 2912, 2685, 880, 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 463: 880, 576: 4884, 653: 4883, 2674, 2675, 2673, 844: 6508}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 5640, 2674, 2675, 2673, 863: 6506}, - {98: 6505}, + {2303, 2303, 7: 5643}, + {461: 6501}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5641, 2675, 2676, 2674, 861: 6502}, + {2297, 2297, 7: 5643, 463: 4868, 790: 6503}, + {2304, 2304}, // 4045 - {98: 6504}, - {2297, 2297, 463: 2297}, - {2298, 2298, 463: 2298}, - {2296, 2296, 7: 5642, 463: 4867, 792: 6507}, - {2300, 2300}, + {2300, 2300, 7: 4628, 98: 6507, 159: 6509, 463: 2300, 637: 6506, 639: 6508, 963: 6505}, + {2297, 2297, 463: 4868, 790: 6516}, + {880, 880, 2911, 2759, 2795, 2913, 2686, 880, 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 463: 880, 576: 4885, 651: 4884, 2675, 2676, 2674, 842: 6514}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 5641, 2675, 2676, 2674, 861: 6512}, + {98: 6511}, // 4050 - {2296, 2296, 7: 4886, 463: 4867, 792: 6509}, - {2305, 2305}, - {2306, 2306}, - {2296, 2296, 7: 4886, 463: 4867, 792: 6512}, - {2308, 2308}, + {98: 6510}, + {2298, 2298, 463: 2298}, + {2299, 2299, 463: 2299}, + {2297, 2297, 7: 5643, 463: 4868, 790: 6513}, + {2301, 2301}, // 4055 - {2296, 2296, 463: 4867, 792: 6514}, + {2297, 2297, 7: 4887, 463: 4868, 790: 6515}, + {2306, 2306}, + {2307, 2307}, + {2297, 2297, 7: 4887, 463: 4868, 790: 6518}, {2309, 2309}, - {559: 6520}, - {482: 6518}, - {559: 2311}, // 4060 - {480: 6519, 559: 2312}, - {559: 2310}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6521}, - {480: 5259, 544: 894, 639: 894, 649: 894, 847: 6522}, - {544: 6525, 639: 6524, 649: 6526, 1114: 6523}, + {2297, 2297, 463: 4868, 790: 6520}, + {2310, 2310}, + {559: 6526}, + {482: 6524}, + {559: 2312}, // 4065 - {2317, 2317}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6533, 2674, 2675, 2673}, - {462: 3794, 827: 6528}, - {462: 3794, 827: 5776, 959: 6527}, - {2314, 2314, 7: 5777}, + {480: 6525, 559: 2313}, + {559: 2311}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6527}, + {480: 5260, 544: 894, 637: 894, 647: 894, 845: 6528}, + {544: 6531, 637: 6530, 647: 6532, 1113: 6529}, // 4070 - {493: 6529}, - {462: 3794, 827: 6530}, - {86: 6531}, - {496: 2648, 725: 3948, 751: 6532}, - {2315, 2315}, + {2318, 2318}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6539, 2675, 2676, 2674}, + {462: 3795, 825: 6534}, + {462: 3795, 825: 5777, 957: 6533}, + {2315, 2315, 7: 5778}, // 4075 - {544: 6525, 649: 6526, 1114: 6534}, + {494: 6535}, + {462: 3795, 825: 6536}, + {86: 6537}, + {497: 2649, 721: 3949, 749: 6538}, {2316, 2316}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6536}, - {2319, 2319, 640: 6538, 1193: 6537}, - {2320, 2320}, // 4080 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6539, 2674, 2675, 2673}, - {2318, 2318}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 649: 6541, 653: 3818, 2674, 2675, 2673, 727: 6542}, - {252: 6544}, - {2322, 2322, 496: 2648, 725: 3948, 751: 6543}, - // 4085 + {544: 6531, 647: 6532, 1113: 6540}, + {2317, 2317}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6542}, + {2320, 2320, 638: 6544, 1192: 6543}, {2321, 2321}, - {496: 2648, 725: 3948, 751: 6545}, - {2323, 2323}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6557, 1129: 6556, 1299: 6555}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 6550, 1134: 6549, 1304: 6548}, + // 4085 + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6545, 2675, 2676, 2674}, + {2319, 2319}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 647: 6547, 651: 3819, 2675, 2676, 2674, 725: 6548}, + {252: 6550}, + {2323, 2323, 497: 2649, 721: 3949, 749: 6549}, // 4090 - {2327, 2327, 7: 6553}, - {2326, 2326, 7: 2326}, - {640: 6551}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 6552}, - {2324, 2324, 7: 2324}, + {2322, 2322}, + {497: 2649, 721: 3949, 749: 6551}, + {2324, 2324}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6563, 1128: 6562, 1297: 6561}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 6556, 1133: 6555, 1302: 6554}, // 4095 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 6550, 1134: 6554}, + {2328, 2328, 7: 6559}, + {2327, 2327, 7: 2327}, + {638: 6557}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 6558}, {2325, 2325, 7: 2325}, - {2331, 2331, 7: 6560}, - {2330, 2330, 7: 2330}, - {640: 6558}, // 4100 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6559}, - {2328, 2328, 7: 2328}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6557, 1129: 6561}, - {2329, 2329, 7: 2329}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 2118, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 653: 5190, 2674, 2675, 2673, 766: 6419, 780: 6418, 803: 6611, 836: 6421, 919: 6612}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 6556, 1133: 6560}, + {2326, 2326, 7: 2326}, + {2332, 2332, 7: 6566}, + {2331, 2331, 7: 2331}, + {638: 6564}, // 4105 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 6601, 2674, 2675, 2673}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 464: 1817, 555: 4342, 558: 1817, 767: 6590}, - {266: 6584, 1212: 6583}, - {158: 6579}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6568}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6565}, + {2329, 2329, 7: 2329}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6563, 1128: 6567}, + {2330, 2330, 7: 2330}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 2118, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 651: 5191, 2675, 2676, 2674, 764: 6420, 778: 6419, 801: 6617, 834: 6423, 917: 6618}, // 4110 - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 3818, 2674, 2675, 2673, 727: 6569}, - {70: 6251, 6248, 6254, 6255, 6256, 6249, 6247, 6257, 6253, 6250, 6573, 647: 6252, 900: 6572, 971: 6571, 1147: 6570}, - {25, 25, 70: 6251, 6248, 6254, 6255, 6256, 6249, 6247, 6257, 6253, 6250, 6573, 647: 6252, 900: 6572, 971: 6578}, - {24, 24, 70: 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 647: 24}, - {22, 22, 70: 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 647: 22}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 6607, 2675, 2676, 2674}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 464: 1817, 555: 4343, 558: 1817, 765: 6596}, + {266: 6590, 1211: 6589}, + {158: 6585}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6574}, // 4115 - {21, 21, 70: 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 463: 6575, 474: 1999, 1999, 487: 4172, 496: 1999, 647: 21, 726: 6574}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6577}, - {474: 3952, 3951, 496: 2648, 725: 3948, 751: 3950, 801: 6576}, - {19, 19, 70: 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 647: 19}, - {20, 20, 70: 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 647: 20}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 3819, 2675, 2676, 2674, 725: 6575}, + {70: 6252, 6249, 6255, 6256, 6257, 6250, 6248, 6258, 6254, 6251, 6579, 645: 6253, 898: 6578, 969: 6577, 1146: 6576}, + {25, 25, 70: 6252, 6249, 6255, 6256, 6257, 6250, 6248, 6258, 6254, 6251, 6579, 645: 6253, 898: 6578, 969: 6584}, + {24, 24, 70: 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 645: 24}, + {22, 22, 70: 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 645: 22}, // 4120 - {23, 23, 70: 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 647: 23}, - {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4342, 767: 6580}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 3280, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 653: 4862, 2674, 2675, 2673, 872: 6581}, - {86: 6343, 88: 6348, 6350, 6344, 6349, 6352, 6346, 6342, 6347, 6351, 6345, 860: 6340, 1084: 6582}, - {51, 51, 7: 6376, 86: 6343, 88: 6348, 6350, 6344, 6349, 6352, 6346, 6342, 6347, 6351, 6345, 860: 6375}, + {21, 21, 70: 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 463: 6581, 474: 1999, 1999, 487: 4173, 497: 1999, 645: 21, 724: 6580}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6583}, + {474: 3953, 3952, 497: 2649, 721: 3949, 749: 3951, 799: 6582}, + {19, 19, 70: 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 645: 19}, + {20, 20, 70: 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 645: 20}, // 4125 - {229, 229}, - {386: 6585}, - {228, 228, 70: 6586}, - {151: 6587}, - {461: 6588}, + {23, 23, 70: 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 645: 23}, + {2: 1817, 1817, 1817, 1817, 1817, 8: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 51: 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 1817, 555: 4343, 765: 6586}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 3279, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 651: 4863, 2675, 2676, 2674, 870: 6587}, + {86: 6344, 88: 6349, 6351, 6345, 6350, 6353, 6347, 6343, 6348, 6352, 6346, 858: 6341, 1083: 6588}, + {51, 51, 7: 6377, 86: 6344, 88: 6349, 6351, 6345, 6350, 6353, 6347, 6343, 6348, 6352, 6346, 858: 6376}, // 4130 - {197: 6589}, - {227, 227}, - {2: 2910, 2758, 2794, 2912, 2685, 8: 2731, 2686, 2817, 2929, 2922, 3273, 3268, 2797, 3073, 2799, 2773, 2720, 2709, 2717, 2742, 2801, 2802, 2906, 2796, 2930, 3032, 3031, 2684, 2795, 2798, 2809, 2749, 2753, 2805, 2915, 2764, 2843, 2682, 2683, 2842, 2914, 2681, 2927, 2887, 2998, 2763, 2766, 51: 2981, 2978, 2970, 2982, 2985, 2986, 2983, 2987, 2988, 2984, 2977, 2989, 2972, 2973, 2976, 2979, 2980, 2990, 3276, 2829, 2767, 2957, 2956, 2958, 2953, 2952, 2959, 2954, 2955, 2759, 2872, 2942, 3005, 2940, 3006, 3044, 2941, 3123, 3127, 3116, 3126, 3128, 3119, 3124, 3125, 3129, 3122, 2700, 2832, 2771, 3266, 2694, 2837, 2928, 3277, 3270, 2729, 3289, 2939, 2772, 3272, 3287, 3288, 3286, 3282, 2931, 2932, 2933, 2934, 2935, 2936, 2938, 3278, 2857, 2768, 2861, 2862, 2863, 2864, 2853, 2881, 2924, 2883, 2702, 2882, 2744, 3003, 2834, 2873, 2739, 2792, 2948, 2854, 2813, 2703, 2708, 2719, 2734, 2943, 2816, 2761, 2783, 2689, 2833, 2718, 2738, 3104, 2992, 3077, 2869, 2781, 6591, 3265, 2748, 3075, 2752, 2760, 2782, 2993, 2693, 2711, 3269, 2732, 2810, 2811, 2962, 2890, 2999, 3000, 2964, 2828, 3001, 2920, 3072, 3026, 2960, 2762, 2860, 3274, 2918, 2820, 2679, 2825, 2715, 2716, 2826, 2723, 2733, 2736, 2724, 2946, 2971, 2786, 2885, 3074, 2852, 2823, 2880, 2923, 2812, 3027, 2770, 3037, 3275, 2919, 3008, 2968, 2830, 2891, 2692, 3009, 3012, 2698, 2994, 3013, 3285, 2704, 2705, 2893, 3055, 3015, 2889, 2713, 3017, 2902, 2926, 2913, 2714, 3019, 2921, 2727, 2951, 3111, 2737, 2740, 2903, 2949, 3064, 2944, 3065, 2897, 3021, 3020, 2947, 3004, 2835, 3290, 3022, 3023, 2839, 2895, 3024, 3002, 2756, 2757, 2868, 2974, 2870, 3078, 3025, 2916, 2917, 2858, 2765, 2899, 3040, 3028, 2680, 3087, 2898, 3094, 3095, 3096, 3097, 3099, 3098, 3100, 3101, 3039, 2778, 2676, 2677, 2950, 2967, 2687, 2969, 2995, 2690, 2691, 3053, 3010, 3011, 2695, 2879, 2696, 2697, 2866, 3281, 3014, 2814, 2701, 2706, 2707, 3016, 3018, 3059, 3060, 2721, 2722, 2836, 2726, 2886, 3105, 2728, 2896, 3271, 2831, 2807, 3034, 2904, 2925, 2888, 2822, 3066, 2874, 2892, 2937, 2745, 2743, 2819, 2905, 2800, 2961, 2875, 2803, 2804, 3291, 2838, 2747, 2769, 3041, 3106, 2750, 2908, 2911, 2963, 2997, 3042, 3007, 2848, 2849, 2855, 3070, 3045, 3071, 2945, 3046, 2975, 2878, 2818, 2909, 2867, 3033, 3030, 3029, 3079, 2894, 2996, 2907, 3091, 3036, 2876, 2774, 2775, 3038, 3114, 3102, 2900, 2779, 2808, 2815, 2877, 3120, 2784, 3043, 2884, 3047, 2789, 3048, 3049, 3267, 3050, 3051, 3052, 3107, 3054, 3056, 3057, 3058, 2725, 2871, 3108, 2841, 3061, 2730, 3115, 3294, 3063, 3298, 3297, 3292, 3117, 3118, 3068, 3067, 2746, 3069, 3076, 2847, 2754, 2755, 2991, 2865, 3283, 3284, 3293, 2859, 2790, 2901, 2821, 2824, 3109, 3083, 3084, 3085, 3086, 3110, 3080, 3081, 3082, 2840, 3035, 3295, 3296, 3103, 3088, 3089, 3090, 3121, 3279, 464: 3389, 558: 5091, 653: 3390, 2674, 2675, 2673, 728: 5090, 762: 5108, 877: 5109, 909: 6592}, - {1670, 1670, 7: 1670, 13: 1670, 46: 1670, 141: 1670, 462: 6596, 1670, 557: 1670, 651: 1670, 657: 1670}, - {215, 215, 7: 5111, 13: 215, 46: 215, 463: 215, 651: 5155, 944: 5154, 6593}, + {229, 229}, + {386: 6591}, + {228, 228, 70: 6592}, + {151: 6593}, + {461: 6594}, // 4135 - {223, 223, 13: 223, 46: 223, 463: 6293, 992: 6594}, - {202, 202, 13: 6310, 46: 6308, 937: 6309, 6307, 1082: 6306, 6595}, - {231, 231}, - {50: 6597}, - {141: 6598}, + {198: 6595}, + {227, 227}, + {2: 2911, 2759, 2795, 2913, 2686, 8: 2732, 2687, 2818, 2930, 2923, 3272, 3267, 2798, 3074, 2800, 2774, 2718, 2721, 2710, 2743, 2802, 2803, 2907, 2797, 2931, 3033, 3032, 2685, 2796, 2799, 2810, 2750, 2754, 2806, 2916, 2765, 2844, 2683, 2684, 2843, 2915, 2682, 2928, 2888, 2999, 2764, 2767, 51: 2982, 2979, 2971, 2983, 2986, 2987, 2984, 2988, 2989, 2985, 2978, 2990, 2973, 2974, 2977, 2980, 2981, 2991, 3275, 2830, 2768, 2958, 2957, 2959, 2954, 2953, 2960, 2955, 2956, 2760, 2873, 2943, 3006, 2941, 3007, 3045, 2942, 3124, 3128, 3117, 3127, 3129, 3120, 3125, 3126, 3130, 3123, 2701, 2833, 2772, 3265, 2695, 2838, 2929, 3276, 3269, 2730, 3288, 2940, 2773, 3271, 3286, 3287, 3285, 3281, 2932, 2933, 2934, 2935, 2936, 2937, 2939, 3277, 2858, 2769, 2862, 2863, 2864, 2865, 2854, 2882, 2925, 2884, 2703, 2883, 2745, 3004, 2835, 2874, 2740, 2793, 2949, 2855, 2814, 2704, 2709, 2720, 2735, 2944, 2817, 2762, 2784, 2690, 2834, 2719, 2739, 3105, 2993, 3078, 2870, 2782, 6597, 3264, 2749, 3076, 2753, 2761, 2783, 2994, 2694, 2712, 3268, 2733, 2811, 2812, 2947, 2963, 2891, 3000, 3001, 2965, 2829, 3002, 2921, 3073, 3027, 2961, 2763, 2861, 3273, 2919, 2821, 2680, 2826, 2716, 2717, 2827, 2724, 2734, 2737, 2725, 2972, 2787, 2886, 3075, 2853, 2824, 2881, 2924, 2813, 3028, 2771, 3038, 3274, 2920, 3009, 2969, 2831, 2892, 2693, 3010, 3013, 2699, 2995, 3014, 3284, 2705, 2706, 2894, 3056, 3016, 2890, 2714, 3018, 2903, 2927, 2914, 2715, 3020, 2922, 2728, 2952, 3112, 2738, 2741, 2904, 2950, 3065, 2945, 3066, 2898, 3022, 3021, 2948, 3005, 2836, 3289, 3023, 3024, 2840, 2896, 3025, 3003, 2757, 2758, 2869, 2946, 2975, 2871, 3079, 3026, 2917, 2918, 2859, 2766, 2900, 3041, 3029, 2681, 3088, 2899, 3039, 3095, 3096, 3097, 3098, 3100, 3099, 3101, 3102, 3040, 2779, 2677, 2678, 2951, 2968, 2688, 2970, 2996, 2691, 2692, 3054, 3011, 3012, 2696, 2880, 2697, 2698, 2867, 3280, 3015, 2815, 2702, 2707, 2708, 3017, 3019, 3060, 3061, 2722, 2723, 2837, 2727, 2887, 3106, 2729, 2897, 3270, 2832, 2808, 3035, 2905, 2926, 2889, 2823, 3067, 2875, 2893, 2938, 2746, 2744, 2820, 2906, 2801, 2962, 2876, 2804, 2805, 3290, 2839, 2748, 2770, 3042, 3107, 2751, 2909, 2912, 2964, 2998, 3043, 3008, 2849, 2850, 2856, 3071, 3046, 3072, 3047, 2976, 2879, 2819, 2910, 2868, 3034, 3031, 3030, 3080, 2895, 2997, 2908, 3092, 3037, 2877, 2775, 2776, 3115, 3103, 2901, 2780, 2809, 2816, 2878, 3121, 2785, 3044, 2885, 3293, 2790, 3049, 3050, 3266, 3051, 3052, 3053, 3108, 3055, 3057, 3058, 3059, 2726, 2872, 3109, 2842, 3062, 2731, 3116, 3294, 3064, 3299, 3298, 3291, 3118, 3119, 3069, 3068, 2747, 3070, 3077, 2848, 2755, 2756, 2992, 2866, 3282, 3283, 3292, 2860, 2791, 2902, 2822, 2825, 3110, 3084, 3085, 3086, 3087, 3111, 3295, 3082, 3083, 2841, 3036, 3296, 3297, 3104, 3089, 3090, 3091, 3122, 3278, 464: 3390, 558: 5092, 651: 3391, 2675, 2676, 2674, 726: 5091, 760: 5109, 875: 5110, 907: 6598}, + {1670, 1670, 7: 1670, 13: 1670, 46: 1670, 141: 1670, 462: 6602, 1670, 557: 1670, 649: 1670, 655: 1670}, + {215, 215, 7: 5112, 13: 215, 46: 215, 463: 215, 649: 5156, 942: 5155, 6599}, // 4140 - {649: 6599}, - {464: 5124, 879: 6600}, - {230, 230}, - {1910, 1910, 15: 1910, 47: 1910, 49: 1910, 51: 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 137: 6327, 461: 1910, 495: 6326, 643: 1910, 1025: 6602}, - {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 643: 1967, 885: 6603}, + {223, 223, 13: 223, 46: 223, 463: 6294, 990: 6600}, + {202, 202, 13: 6311, 46: 6309, 935: 6310, 6308, 1081: 6307, 6601}, + {231, 231}, + {50: 6603}, + {141: 6604}, // 4145 - {1904, 1904, 15: 6019, 47: 5995, 49: 6015, 51: 6008, 5998, 5994, 6002, 6006, 6018, 6001, 6007, 6005, 6003, 6016, 6009, 5997, 6017, 5996, 5999, 6000, 6004, 6605, 461: 6010, 643: 6020, 881: 6012, 6011, 6014, 5993, 886: 6013, 1206: 6604}, - {1919, 1919}, - {200: 6607, 641: 6606}, - {545, 545, 559: 5966, 953: 6609}, - {545, 545, 559: 5966, 953: 6608}, + {647: 6605}, + {464: 5125, 877: 6606}, + {230, 230}, + {1910, 1910, 15: 1910, 47: 1910, 49: 1910, 51: 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 1910, 137: 6328, 461: 1910, 496: 6327, 641: 1910, 1023: 6608}, + {1967, 1967, 15: 1967, 47: 1967, 49: 1967, 51: 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 1967, 461: 1967, 641: 1967, 883: 6609}, // 4150 + {1904, 1904, 15: 6020, 47: 5996, 49: 6016, 51: 6009, 5999, 5995, 6003, 6007, 6019, 6002, 6008, 6006, 6004, 6017, 6010, 5998, 6018, 5997, 6000, 6001, 6005, 6611, 461: 6011, 641: 6021, 879: 6013, 6012, 6015, 5994, 884: 6014, 1205: 6610}, + {1919, 1919}, + {201: 6613, 639: 6612}, + {545, 545, 559: 5967, 951: 6615}, + {545, 545, 559: 5967, 951: 6614}, + // 4155 {1902, 1902}, {1903, 1903}, - {14: 1386, 16: 1386, 21: 1386, 158: 4855, 468: 1386, 473: 1386, 494: 1386, 637: 1386}, - {14: 2118, 16: 2118, 21: 2118, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 766: 6419, 780: 6418, 836: 6421, 919: 6613}, - {2136, 2136, 14: 2118, 16: 2118, 21: 2118, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 766: 6419, 780: 6418, 836: 6422}, - // 4155 - {2137, 2137, 14: 2118, 16: 2118, 21: 2118, 468: 4131, 473: 2118, 494: 2118, 637: 2118, 766: 6419, 780: 6418, 836: 6422}, - {1997, 1997, 47: 2487, 69: 2602, 71: 2468, 80: 2498, 145: 2470, 151: 2496, 153: 2467, 166: 2492, 198: 2517, 205: 2614, 208: 2463, 216: 2516, 2483, 2469, 233: 2495, 238: 2473, 241: 2493, 243: 2464, 245: 2499, 263: 2485, 267: 2484, 274: 2497, 276: 2465, 279: 2486, 290: 2478, 462: 2507, 2506, 485: 2610, 491: 2505, 495: 2491, 501: 2515, 514: 2605, 518: 2481, 556: 2490, 559: 2504, 636: 2500, 639: 2613, 643: 2466, 2604, 652: 2461, 659: 2472, 664: 2471, 669: 2514, 676: 2462, 699: 2511, 732: 2474, 741: 2513, 2501, 2502, 2503, 2512, 2510, 2509, 2508, 752: 2584, 2583, 2477, 763: 2603, 2475, 768: 2567, 2578, 771: 2594, 781: 2476, 785: 2533, 797: 2608, 810: 2521, 832: 2528, 835: 2531, 841: 2606, 846: 2570, 850: 2575, 2585, 2488, 918: 2540, 922: 2479, 957: 2609, 964: 2519, 966: 2520, 2523, 2524, 970: 2526, 972: 2525, 974: 2522, 976: 2527, 2529, 2530, 980: 2489, 2566, 983: 2536, 993: 2544, 2537, 2538, 2539, 2545, 2543, 2546, 2547, 1002: 2542, 2541, 1005: 2532, 2494, 2480, 2548, 2560, 2549, 2550, 2551, 2553, 2557, 2554, 2558, 2559, 2552, 2556, 2555, 1022: 2518, 1026: 2534, 2535, 2482, 1032: 2562, 2561, 1036: 2564, 2565, 2563, 1041: 2600, 2568, 1049: 2612, 2611, 2569, 1056: 2571, 1058: 2597, 1085: 2572, 2573, 1088: 2574, 1090: 2579, 1093: 2576, 2577, 1096: 2599, 2580, 2607, 2582, 2581, 1106: 2587, 2586, 2590, 1110: 2591, 1112: 2598, 1115: 2588, 6615, 1120: 2589, 1131: 2592, 2593, 2596, 1135: 2595}, + {14: 1386, 16: 1386, 19: 1386, 158: 4856, 468: 1386, 473: 1386, 493: 1386, 495: 1386, 635: 1386}, + {14: 2118, 16: 2118, 19: 2118, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 764: 6420, 778: 6419, 834: 6423, 917: 6619}, + {2137, 2137, 14: 2118, 16: 2118, 19: 2118, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 764: 6420, 778: 6419, 834: 6424}, + // 4160 + {2138, 2138, 14: 2118, 16: 2118, 19: 2118, 468: 4132, 473: 2118, 493: 6421, 495: 2118, 635: 2118, 764: 6420, 778: 6419, 834: 6424}, + {1997, 1997, 47: 2488, 69: 2603, 71: 2469, 80: 2499, 145: 2471, 151: 2497, 153: 2468, 166: 2493, 199: 2518, 205: 2615, 208: 2464, 216: 2517, 2484, 2470, 233: 2496, 238: 2474, 241: 2494, 243: 2465, 245: 2500, 263: 2486, 268: 2485, 275: 2498, 277: 2466, 280: 2487, 292: 2479, 462: 2508, 2507, 485: 2611, 491: 2506, 493: 2516, 496: 2492, 514: 2606, 518: 2482, 556: 2491, 559: 2505, 634: 2501, 637: 2614, 641: 2467, 2605, 650: 2462, 657: 2473, 662: 2472, 667: 2515, 674: 2463, 697: 2512, 730: 2475, 739: 2514, 2502, 2503, 2504, 2513, 2511, 2510, 2509, 750: 2585, 2584, 2478, 761: 2604, 2476, 766: 2568, 2579, 769: 2595, 779: 2477, 783: 2534, 795: 2609, 808: 2522, 830: 2529, 833: 2532, 839: 2607, 844: 2571, 848: 2576, 2586, 2489, 916: 2541, 920: 2480, 955: 2610, 962: 2520, 964: 2521, 2524, 2525, 968: 2527, 970: 2526, 972: 2523, 974: 2528, 2530, 2531, 978: 2490, 2567, 981: 2537, 991: 2545, 2538, 2539, 2540, 2546, 2544, 2547, 2548, 1000: 2543, 2542, 1003: 2533, 2495, 2481, 2549, 2561, 2550, 2551, 2552, 2554, 2558, 2555, 2559, 2560, 2553, 2557, 2556, 1020: 2519, 1024: 2535, 2536, 2483, 1030: 2563, 2562, 1034: 2565, 2566, 2564, 1039: 2601, 2569, 1047: 2613, 2612, 2570, 1054: 2572, 1057: 2598, 1084: 2573, 2574, 1087: 2575, 1089: 2580, 1092: 2577, 2578, 1095: 2600, 2581, 2608, 2583, 2582, 1105: 2588, 2587, 2591, 1109: 2592, 1111: 2599, 1114: 2589, 6621, 1119: 2590, 1130: 2593, 2594, 2597, 1134: 2596}, {433, 433}, } ) @@ -11047,7 +11048,7 @@ func yylex1(yylex yyLexer, lval *yySymType) (n int) { } func yyParse(yylex yyLexer, parser *Parser) int { - const yyError = 1336 + const yyError = 1334 yyEx, _ := yylex.(yyLexerEx) var yyn int @@ -13078,18 +13079,29 @@ yynewstate: } } case 331: + { + tiflashReplicaSpec := &ast.TiFlashReplicaSpec{ + Count: yyS[yypt-1].item.(uint64), + Labels: yyS[yypt-0].item.([]string), + } + parser.yyVAL.item = &ast.DatabaseOption{ + Tp: ast.DatabaseSetTiFlashReplica, + TiFlashReplica: tiflashReplicaSpec, + } + } + case 332: { parser.yyVAL.item = []*ast.DatabaseOption{} } - case 333: + case 334: { parser.yyVAL.item = []*ast.DatabaseOption{yyS[yypt-0].item.(*ast.DatabaseOption)} } - case 334: + case 335: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.DatabaseOption), yyS[yypt-0].item.(*ast.DatabaseOption)) } - case 335: + case 336: { stmt := yyS[yypt-6].item.(*ast.CreateTableStmt) stmt.Table = yyS[yypt-7].item.(*ast.TableName) @@ -13110,7 +13122,7 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 336: + case 337: { tmp := &ast.CreateTableStmt{ Table: yyS[yypt-2].item.(*ast.TableName), @@ -13127,23 +13139,23 @@ yynewstate: } parser.yyVAL.statement = tmp } - case 337: + case 338: { parser.yyVAL.item = nil } - case 338: + case 339: { parser.yyVAL.item = true } - case 339: + case 340: { parser.yyVAL.item = false } - case 342: + case 343: { parser.yyVAL.item = nil } - case 343: + case 344: { method := yyS[yypt-3].item.(*ast.PartitionMethod) method.Num = yyS[yypt-2].item.(uint64) @@ -13160,7 +13172,7 @@ yynewstate: } parser.yyVAL.item = opt } - case 344: + case 345: { keyAlgorithm, _ := yyS[yypt-3].item.(*ast.PartitionKeyAlgorithm) parser.yyVAL.item = &ast.PartitionMethod{ @@ -13170,7 +13182,7 @@ yynewstate: KeyAlgorithm: keyAlgorithm, } } - case 345: + case 346: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeHash, @@ -13178,11 +13190,11 @@ yynewstate: Expr: yyS[yypt-1].expr.(ast.ExprNode), } } - case 346: + case 347: { parser.yyVAL.item = nil } - case 347: + case 348: { tp := getUint64FromNUM(yyS[yypt-0].item) if tp != 1 && tp != 2 { @@ -13193,35 +13205,35 @@ yynewstate: Type: tp, } } - case 349: + case 350: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeRange, Expr: yyS[yypt-1].expr.(ast.ExprNode), } } - case 350: + case 351: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeRange, ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), } } - case 351: + case 352: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeList, Expr: yyS[yypt-1].expr.(ast.ExprNode), } } - case 352: + case 353: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeList, ColumnNames: yyS[yypt-1].item.([]*ast.ColumnName), } } - case 353: + case 354: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeSystemTime, @@ -13229,38 +13241,38 @@ yynewstate: Unit: yyS[yypt-0].item.(ast.TimeUnitType), } } - case 354: + case 355: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeSystemTime, Limit: yyS[yypt-0].item.(uint64), } } - case 355: + case 356: { parser.yyVAL.item = &ast.PartitionMethod{ Tp: model.PartitionTypeSystemTime, } } - case 356: + case 357: { parser.yyVAL.ident = "" } - case 358: + case 359: { parser.yyVAL.item = nil } - case 359: + case 360: { method := yyS[yypt-1].item.(*ast.PartitionMethod) method.Num = yyS[yypt-0].item.(uint64) parser.yyVAL.item = method } - case 360: + case 361: { parser.yyVAL.item = uint64(0) } - case 361: + case 362: { res := yyS[yypt-0].item.(uint64) if res == 0 { @@ -13269,11 +13281,11 @@ yynewstate: } parser.yyVAL.item = res } - case 362: + case 363: { parser.yyVAL.item = uint64(0) } - case 363: + case 364: { res := yyS[yypt-0].item.(uint64) if res == 0 { @@ -13282,23 +13294,23 @@ yynewstate: } parser.yyVAL.item = res } - case 364: + case 365: { parser.yyVAL.item = nil } - case 365: + case 366: { parser.yyVAL.item = yyS[yypt-1].item.([]*ast.PartitionDefinition) } - case 366: + case 367: { parser.yyVAL.item = []*ast.PartitionDefinition{yyS[yypt-0].item.(*ast.PartitionDefinition)} } - case 367: + case 368: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.PartitionDefinition), yyS[yypt-0].item.(*ast.PartitionDefinition)) } - case 368: + case 369: { parser.yyVAL.item = &ast.PartitionDefinition{ Name: model.NewCIStr(yyS[yypt-3].ident), @@ -13307,80 +13319,80 @@ yynewstate: Sub: yyS[yypt-0].item.([]*ast.SubPartitionDefinition), } } - case 369: + case 370: { parser.yyVAL.item = make([]*ast.SubPartitionDefinition, 0) } - case 370: + case 371: { parser.yyVAL.item = yyS[yypt-1].item } - case 371: + case 372: { parser.yyVAL.item = []*ast.SubPartitionDefinition{yyS[yypt-0].item.(*ast.SubPartitionDefinition)} } - case 372: + case 373: { list := yyS[yypt-2].item.([]*ast.SubPartitionDefinition) parser.yyVAL.item = append(list, yyS[yypt-0].item.(*ast.SubPartitionDefinition)) } - case 373: + case 374: { parser.yyVAL.item = &ast.SubPartitionDefinition{ Name: model.NewCIStr(yyS[yypt-1].ident), Options: yyS[yypt-0].item.([]*ast.TableOption), } } - case 374: + case 375: { parser.yyVAL.item = make([]*ast.TableOption, 0) } - case 375: + case 376: { list := yyS[yypt-1].item.([]*ast.TableOption) parser.yyVAL.item = append(list, yyS[yypt-0].item.(*ast.TableOption)) } - case 376: + case 377: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionComment, StrValue: yyS[yypt-0].ident} } - case 377: + case 378: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionEngine, StrValue: yyS[yypt-0].ident} } - case 378: + case 379: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionEngine, StrValue: yyS[yypt-0].ident} } - case 379: + case 380: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionInsertMethod, StrValue: yyS[yypt-0].ident} } - case 380: + case 381: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionDataDirectory, StrValue: yyS[yypt-0].ident} } - case 381: + case 382: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionIndexDirectory, StrValue: yyS[yypt-0].ident} } - case 382: + case 383: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionMaxRows, UintValue: yyS[yypt-0].item.(uint64)} } - case 383: + case 384: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionMinRows, UintValue: yyS[yypt-0].item.(uint64)} } - case 384: + case 385: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionTablespace, StrValue: yyS[yypt-0].ident} } - case 385: + case 386: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionNodegroup, UintValue: yyS[yypt-0].item.(uint64)} } - case 386: + case 387: { placementOptions := yyS[yypt-0].item.(*ast.PlacementOption) parser.yyVAL.item = &ast.TableOption{ @@ -13390,27 +13402,27 @@ yynewstate: UintValue: placementOptions.UintValue, } } - case 387: + case 388: { parser.yyVAL.item = &ast.PartitionDefinitionClauseNone{} } - case 388: + case 389: { parser.yyVAL.item = &ast.PartitionDefinitionClauseLessThan{ Exprs: []ast.ExprNode{&ast.MaxValueExpr{}}, } } - case 389: + case 390: { parser.yyVAL.item = &ast.PartitionDefinitionClauseLessThan{ Exprs: yyS[yypt-1].item.([]ast.ExprNode), } } - case 390: + case 391: { parser.yyVAL.item = &ast.PartitionDefinitionClauseIn{} } - case 391: + case 392: { exprs := yyS[yypt-1].item.([]ast.ExprNode) values := make([][]ast.ExprNode, 0, len(exprs)) @@ -13423,33 +13435,29 @@ yynewstate: } parser.yyVAL.item = &ast.PartitionDefinitionClauseIn{Values: values} } - case 392: - { - parser.yyVAL.item = &ast.PartitionDefinitionClauseHistory{Current: false} - } case 393: { - parser.yyVAL.item = &ast.PartitionDefinitionClauseHistory{Current: true} + parser.yyVAL.item = &ast.PartitionDefinitionClauseHistory{Current: false} } case 394: { - parser.yyVAL.item = ast.OnDuplicateKeyHandlingError + parser.yyVAL.item = &ast.PartitionDefinitionClauseHistory{Current: true} } case 395: { - parser.yyVAL.item = ast.OnDuplicateKeyHandlingIgnore + parser.yyVAL.item = ast.OnDuplicateKeyHandlingError } case 396: { - parser.yyVAL.item = ast.OnDuplicateKeyHandlingReplace + parser.yyVAL.item = ast.OnDuplicateKeyHandlingIgnore } - case 399: + case 397: { - parser.yyVAL.item = &ast.CreateTableStmt{} + parser.yyVAL.item = ast.OnDuplicateKeyHandlingReplace } case 400: { - parser.yyVAL.item = &ast.CreateTableStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + parser.yyVAL.item = &ast.CreateTableStmt{} } case 401: { @@ -13460,6 +13468,10 @@ yynewstate: parser.yyVAL.item = &ast.CreateTableStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} } case 403: + { + parser.yyVAL.item = &ast.CreateTableStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + } + case 404: { var sel ast.ResultSetNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -13472,7 +13484,7 @@ yynewstate: } parser.yyVAL.item = &ast.CreateTableStmt{Select: sel} } - case 407: + case 408: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -13485,15 +13497,15 @@ yynewstate: } parser.yyVAL.statement = sel } - case 408: + case 409: { parser.yyVAL.item = yyS[yypt-0].item } - case 409: + case 410: { parser.yyVAL.item = yyS[yypt-1].item } - case 410: + case 411: { startOffset := parser.startOffset(&yyS[yypt-1]) selStmt := yyS[yypt-1].statement.(ast.StmtNode) @@ -13518,17 +13530,13 @@ yynewstate: } parser.yyVAL.statement = x } - case 411: - { - parser.yyVAL.item = false - } case 412: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 413: { - parser.yyVAL.item = model.AlgorithmUndefined + parser.yyVAL.item = true } case 414: { @@ -13536,67 +13544,71 @@ yynewstate: } case 415: { - parser.yyVAL.item = model.AlgorithmMerge + parser.yyVAL.item = model.AlgorithmUndefined } case 416: { - parser.yyVAL.item = model.AlgorithmTemptable + parser.yyVAL.item = model.AlgorithmMerge } case 417: { - parser.yyVAL.item = &auth.UserIdentity{CurrentUser: true} + parser.yyVAL.item = model.AlgorithmTemptable } case 418: { - parser.yyVAL.item = yyS[yypt-0].item + parser.yyVAL.item = &auth.UserIdentity{CurrentUser: true} } case 419: { - parser.yyVAL.item = model.SecurityDefiner + parser.yyVAL.item = yyS[yypt-0].item } case 420: { parser.yyVAL.item = model.SecurityDefiner } case 421: + { + parser.yyVAL.item = model.SecurityDefiner + } + case 422: { parser.yyVAL.item = model.SecurityInvoker } - case 423: + case 424: { parser.yyVAL.item = nil } - case 424: + case 425: { parser.yyVAL.item = yyS[yypt-1].item.([]model.CIStr) } - case 425: + case 426: { parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 426: + case 427: { parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) } - case 427: + case 428: { parser.yyVAL.item = nil } - case 428: + case 429: { parser.yyVAL.item = model.CheckOptionCascaded } - case 429: + case 430: { parser.yyVAL.item = model.CheckOptionLocal } - case 430: + case 431: { parser.yyVAL.statement = &ast.DoStmt{ Exprs: yyS[yypt-0].item.([]ast.ExprNode), } } - case 431: + case 432: { // Single Table tn := yyS[yypt-6].item.(*ast.TableName) @@ -13624,7 +13636,7 @@ yynewstate: parser.yyVAL.statement = x } - case 432: + case 433: { // Multiple Table x := &ast.DeleteStmt{ @@ -13644,7 +13656,7 @@ yynewstate: } parser.yyVAL.statement = x } - case 433: + case 434: { // Multiple Table x := &ast.DeleteStmt{ @@ -13663,23 +13675,23 @@ yynewstate: } parser.yyVAL.statement = x } - case 436: + case 437: { d := yyS[yypt-0].statement.(*ast.DeleteStmt) d.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = d } - case 437: + case 438: { d := yyS[yypt-0].statement.(*ast.DeleteStmt) d.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = d } - case 439: + case 440: { parser.yyVAL.statement = &ast.DropDatabaseStmt{IfExists: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident} } - case 440: + case 441: { var indexLockAndAlgorithm *ast.IndexLockAndAlgorithm if yyS[yypt-0].item != nil { @@ -13690,39 +13702,39 @@ yynewstate: } parser.yyVAL.statement = &ast.DropIndexStmt{IfExists: yyS[yypt-4].item.(bool), IndexName: yyS[yypt-3].ident, Table: yyS[yypt-1].item.(*ast.TableName), LockAlg: indexLockAndAlgorithm} } - case 441: + case 442: { parser.yyVAL.statement = &ast.DropTableStmt{IfExists: yyS[yypt-2].item.(bool), Tables: yyS[yypt-1].item.([]*ast.TableName), IsView: false, TemporaryKeyword: yyS[yypt-4].item.(ast.TemporaryKeyword)} } - case 442: + case 443: { parser.yyVAL.item = ast.TemporaryNone } - case 443: + case 444: { parser.yyVAL.item = ast.TemporaryLocal } - case 444: + case 445: { parser.yyVAL.item = ast.TemporaryGlobal } - case 445: + case 446: { parser.yyVAL.statement = &ast.DropTableStmt{Tables: yyS[yypt-1].item.([]*ast.TableName), IsView: true} } - case 446: + case 447: { parser.yyVAL.statement = &ast.DropTableStmt{IfExists: true, Tables: yyS[yypt-1].item.([]*ast.TableName), IsView: true} } - case 447: + case 448: { parser.yyVAL.statement = &ast.DropUserStmt{IsDropRole: false, IfExists: false, UserList: yyS[yypt-0].item.([]*auth.UserIdentity)} } - case 448: + case 449: { parser.yyVAL.statement = &ast.DropUserStmt{IsDropRole: false, IfExists: true, UserList: yyS[yypt-0].item.([]*auth.UserIdentity)} } - case 449: + case 450: { tmp := make([]*auth.UserIdentity, 0, 10) roleList := yyS[yypt-0].item.([]*auth.RoleIdentity) @@ -13731,7 +13743,7 @@ yynewstate: } parser.yyVAL.statement = &ast.DropUserStmt{IsDropRole: true, IfExists: false, UserList: tmp} } - case 450: + case 451: { tmp := make([]*auth.UserIdentity, 0, 10) roleList := yyS[yypt-0].item.([]*auth.RoleIdentity) @@ -13740,29 +13752,29 @@ yynewstate: } parser.yyVAL.statement = &ast.DropUserStmt{IsDropRole: true, IfExists: true, UserList: tmp} } - case 451: + case 452: { parser.yyVAL.statement = &ast.DropStatsStmt{Table: yyS[yypt-0].item.(*ast.TableName)} } - case 452: + case 453: { parser.yyVAL.statement = &ast.DropStatsStmt{ Table: yyS[yypt-2].item.(*ast.TableName), PartitionNames: yyS[yypt-0].item.([]model.CIStr), } } - case 453: + case 454: { parser.yyVAL.statement = &ast.DropStatsStmt{ Table: yyS[yypt-1].item.(*ast.TableName), IsGlobalStats: true, } } - case 461: + case 462: { parser.yyVAL.statement = nil } - case 462: + case 463: { parser.yyVAL.statement = &ast.TraceStmt{ Stmt: yyS[yypt-0].statement, @@ -13772,7 +13784,7 @@ yynewstate: startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(parser.lexer.client, string(parser.src[startOffset:])) } - case 463: + case 464: { parser.yyVAL.statement = &ast.TraceStmt{ Stmt: yyS[yypt-0].statement, @@ -13782,7 +13794,7 @@ yynewstate: startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(parser.lexer.client, string(parser.src[startOffset:])) } - case 464: + case 465: { parser.yyVAL.statement = &ast.TraceStmt{ Stmt: yyS[yypt-0].statement, @@ -13791,7 +13803,7 @@ yynewstate: startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(parser.lexer.client, string(parser.src[startOffset:])) } - case 465: + case 466: { parser.yyVAL.statement = &ast.TraceStmt{ Stmt: yyS[yypt-0].statement, @@ -13801,7 +13813,7 @@ yynewstate: startOffset := parser.startOffset(&yyS[yypt]) yyS[yypt-0].statement.SetText(parser.lexer.client, string(parser.src[startOffset:])) } - case 469: + case 470: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: &ast.ShowStmt{ @@ -13810,7 +13822,7 @@ yynewstate: }, } } - case 470: + case 471: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: &ast.ShowStmt{ @@ -13820,49 +13832,49 @@ yynewstate: }, } } - case 471: + case 472: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: "row", } } - case 472: + case 473: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: "row", ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 473: + case 474: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: yyS[yypt-3].ident, ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 474: + case 475: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: yyS[yypt-1].ident, } } - case 475: + case 476: { parser.yyVAL.statement = &ast.ExplainForStmt{ Format: yyS[yypt-3].ident, ConnectionID: getUint64FromNUM(yyS[yypt-0].item), } } - case 476: + case 477: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, Format: yyS[yypt-1].ident, } } - case 477: + case 478: { parser.yyVAL.statement = &ast.ExplainStmt{ Stmt: yyS[yypt-0].statement, @@ -13870,7 +13882,7 @@ yynewstate: Analyze: true, } } - case 484: + case 485: { stmt := yyS[yypt-3].item.(*ast.BRIEStmt) stmt.Kind = ast.BRIEKindBackup @@ -13878,7 +13890,7 @@ yynewstate: stmt.Options = yyS[yypt-0].item.([]*ast.BRIEOption) parser.yyVAL.statement = stmt } - case 485: + case 486: { stmt := yyS[yypt-3].item.(*ast.BRIEStmt) stmt.Kind = ast.BRIEKindRestore @@ -13886,110 +13898,110 @@ yynewstate: stmt.Options = yyS[yypt-0].item.([]*ast.BRIEOption) parser.yyVAL.statement = stmt } - case 486: + case 487: { parser.yyVAL.item = &ast.BRIEStmt{} } - case 487: + case 488: { parser.yyVAL.item = &ast.BRIEStmt{Schemas: yyS[yypt-0].item.([]string)} } - case 488: + case 489: { parser.yyVAL.item = &ast.BRIEStmt{Tables: yyS[yypt-0].item.([]*ast.TableName)} } - case 489: + case 490: { parser.yyVAL.item = []string{yyS[yypt-0].ident} } - case 490: + case 491: { parser.yyVAL.item = append(yyS[yypt-2].item.([]string), yyS[yypt-0].ident) } - case 491: + case 492: { parser.yyVAL.item = []*ast.BRIEOption{} } - case 492: + case 493: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.BRIEOption), yyS[yypt-0].item.(*ast.BRIEOption)) } - case 493: + case 494: { parser.yyVAL.item = ast.BRIEOptionConcurrency } - case 494: + case 495: { parser.yyVAL.item = ast.BRIEOptionResume } - case 495: + case 496: { parser.yyVAL.item = ast.BRIEOptionSendCreds } - case 496: + case 497: { parser.yyVAL.item = ast.BRIEOptionOnline } - case 497: + case 498: { parser.yyVAL.item = ast.BRIEOptionCheckpoint } - case 498: + case 499: { parser.yyVAL.item = ast.BRIEOptionSkipSchemaFiles } - case 499: + case 500: { parser.yyVAL.item = ast.BRIEOptionStrictFormat } - case 500: + case 501: { parser.yyVAL.item = ast.BRIEOptionCSVNotNull } - case 501: + case 502: { parser.yyVAL.item = ast.BRIEOptionCSVBackslashEscape } - case 502: + case 503: { parser.yyVAL.item = ast.BRIEOptionCSVTrimLastSeparators } - case 503: + case 504: { parser.yyVAL.item = ast.BRIEOptionTiKVImporter } - case 504: + case 505: { parser.yyVAL.item = ast.BRIEOptionCSVSeparator } - case 505: + case 506: { parser.yyVAL.item = ast.BRIEOptionCSVDelimiter } - case 506: + case 507: { parser.yyVAL.item = ast.BRIEOptionCSVNull } - case 507: + case 508: { parser.yyVAL.item = ast.BRIEOptionBackend } - case 508: + case 509: { parser.yyVAL.item = ast.BRIEOptionOnDuplicate } - case 509: + case 510: { parser.yyVAL.item = ast.BRIEOptionOnDuplicate } - case 510: + case 511: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), UintValue: yyS[yypt-0].item.(uint64), } } - case 511: + case 512: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -14000,21 +14012,21 @@ yynewstate: UintValue: value, } } - case 512: + case 513: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), StrValue: yyS[yypt-0].ident, } } - case 513: + case 514: { parser.yyVAL.item = &ast.BRIEOption{ Tp: yyS[yypt-2].item.(ast.BRIEOptionType), StrValue: strings.ToLower(yyS[yypt-0].ident), } } - case 514: + case 515: { unit, err := yyS[yypt-1].item.(ast.TimeUnitType).Duration() if err != nil { @@ -14027,35 +14039,35 @@ yynewstate: UintValue: yyS[yypt-2].item.(uint64) * uint64(unit), } } - case 515: + case 516: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionBackupTS, StrValue: yyS[yypt-0].ident, } } - case 516: + case 517: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionBackupTSO, UintValue: yyS[yypt-0].item.(uint64), } } - case 517: + case 518: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionLastBackupTS, StrValue: yyS[yypt-0].ident, } } - case 518: + case 519: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionLastBackupTSO, UintValue: yyS[yypt-0].item.(uint64), } } - case 519: + case 520: { // TODO: check overflow? parser.yyVAL.item = &ast.BRIEOption{ @@ -14063,21 +14075,21 @@ yynewstate: UintValue: yyS[yypt-3].item.(uint64) * 1048576, } } - case 520: + case 521: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionCSVHeader, UintValue: ast.BRIECSVHeaderIsColumns, } } - case 521: + case 522: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionCSVHeader, UintValue: yyS[yypt-0].item.(uint64), } } - case 522: + case 523: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -14088,14 +14100,14 @@ yynewstate: UintValue: value, } } - case 523: + case 524: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionChecksum, UintValue: uint64(yyS[yypt-0].item.(ast.BRIEOptionLevel)), } } - case 524: + case 525: { value := uint64(0) if yyS[yypt-0].item.(bool) { @@ -14106,18 +14118,18 @@ yynewstate: UintValue: value, } } - case 525: + case 526: { parser.yyVAL.item = &ast.BRIEOption{ Tp: ast.BRIEOptionAnalyze, UintValue: uint64(yyS[yypt-0].item.(ast.BRIEOptionLevel)), } } - case 526: + case 527: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 527: + case 528: { v, rangeErrMsg := getInt64FromNUM(yyS[yypt-0].item) if len(rangeErrMsg) != 0 { @@ -14126,35 +14138,35 @@ yynewstate: } parser.yyVAL.item = v } - case 529: + case 530: { parser.yyVAL.item = yyS[yypt-0].item.(int64) != 0 } - case 530: + case 531: { parser.yyVAL.item = false } - case 531: + case 532: { parser.yyVAL.item = true } - case 532: + case 533: { parser.yyVAL.item = ast.BRIEOptionLevelOff } - case 533: + case 534: { parser.yyVAL.item = ast.BRIEOptionLevelOptional } - case 534: + case 535: { parser.yyVAL.item = ast.BRIEOptionLevelRequired } - case 535: + case 536: { parser.yyVAL.statement = &ast.PurgeImportStmt{TaskID: getUint64FromNUM(yyS[yypt-0].item)} } - case 536: + case 537: { parser.yyVAL.statement = &ast.CreateImportStmt{ IfNotExists: yyS[yypt-5].item.(bool), @@ -14164,21 +14176,21 @@ yynewstate: Options: yyS[yypt-0].item.([]*ast.BRIEOption), } } - case 537: + case 538: { parser.yyVAL.statement = &ast.StopImportStmt{ IfRunning: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 538: + case 539: { parser.yyVAL.statement = &ast.ResumeImportStmt{ IfNotRunning: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 539: + case 540: { s := &ast.AlterImportStmt{ Name: yyS[yypt-3].ident, @@ -14190,14 +14202,14 @@ yynewstate: } parser.yyVAL.statement = s } - case 540: + case 541: { parser.yyVAL.statement = &ast.DropImportStmt{ IfExists: yyS[yypt-1].item.(bool), Name: yyS[yypt-0].ident, } } - case 541: + case 542: { parser.yyVAL.statement = &ast.ShowImportStmt{ Name: yyS[yypt-2].ident, @@ -14205,73 +14217,73 @@ yynewstate: TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 542: + case 543: { parser.yyVAL.item = false } - case 543: + case 544: { parser.yyVAL.item = true } - case 544: + case 545: { parser.yyVAL.item = false } - case 545: + case 546: { parser.yyVAL.item = true } - case 546: + case 547: { parser.yyVAL.item = false } - case 547: + case 548: { parser.yyVAL.item = true } - case 548: + case 549: { parser.yyVAL.item = ast.ErrorHandleError } - case 549: + case 550: { parser.yyVAL.item = ast.ErrorHandleReplace } - case 550: + case 551: { parser.yyVAL.item = ast.ErrorHandleSkipAll } - case 551: + case 552: { parser.yyVAL.item = ast.ErrorHandleSkipConstraint } - case 552: + case 553: { parser.yyVAL.item = ast.ErrorHandleSkipDuplicate } - case 553: + case 554: { parser.yyVAL.item = ast.ErrorHandleSkipStrict } - case 554: + case 555: { parser.yyVAL.item = nil } - case 555: + case 556: { parser.yyVAL.item = &ast.ImportTruncate{ IsErrorsOnly: false, TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 556: + case 557: { parser.yyVAL.item = &ast.ImportTruncate{ IsErrorsOnly: true, TableNames: yyS[yypt-0].item.([]*ast.TableName), } } - case 557: + case 558: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") @@ -14282,19 +14294,19 @@ yynewstate: Value: yyS[yypt-0].expr, } } - case 558: + case 559: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicOr, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 559: + case 560: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicXor, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 560: + case 561: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LogicAnd, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 561: + case 562: { expr, ok := yyS[yypt-0].expr.(*ast.ExistsSubqueryExpr) if ok { @@ -14304,7 +14316,7 @@ yynewstate: parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not, V: yyS[yypt-0].expr} } } - case 562: + case 563: { parser.yyVAL.expr = &ast.MatchAgainst{ ColumnNames: yyS[yypt-6].item.([]*ast.ColumnName), @@ -14312,87 +14324,87 @@ yynewstate: Modifier: ast.FulltextSearchModifier(yyS[yypt-1].item.(int)), } } - case 563: + case 564: { parser.yyVAL.expr = &ast.IsTruthExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), True: int64(1)} } - case 564: + case 565: { parser.yyVAL.expr = &ast.IsTruthExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), True: int64(0)} } - case 565: + case 566: { /* https://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#operator_is */ parser.yyVAL.expr = &ast.IsNullExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool)} } - case 567: + case 568: { parser.yyVAL.expr = &ast.MaxValueExpr{} } - case 569: + case 570: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode } - case 570: + case 571: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode } - case 571: + case 572: { parser.yyVAL.item = ast.FulltextSearchModifierNaturalLanguageMode | ast.FulltextSearchModifierWithQueryExpansion } - case 572: + case 573: { parser.yyVAL.item = ast.FulltextSearchModifierBooleanMode } - case 573: + case 574: { parser.yyVAL.item = ast.FulltextSearchModifierWithQueryExpansion } - case 578: + case 579: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 579: + case 580: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 580: + case 581: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 581: + case 582: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 582: + case 583: { parser.yyVAL.item = []ast.ExprNode{} } - case 584: + case 585: { parser.yyVAL.item = []ast.ExprNode{} } - case 586: + case 587: { expr := ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) parser.yyVAL.item = []ast.ExprNode{expr} } - case 587: + case 588: { parser.yyVAL.expr = &ast.IsNullExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool)} } - case 588: + case 589: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: yyS[yypt-1].item.(opcode.Op), L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 589: + case 590: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.MultiRows = true parser.yyVAL.expr = &ast.CompareSubqueryExpr{Op: yyS[yypt-2].item.(opcode.Op), L: yyS[yypt-3].expr, R: sq, All: yyS[yypt-1].item.(bool)} } - case 590: + case 591: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") @@ -14404,25 +14416,21 @@ yynewstate: } parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: yyS[yypt-3].item.(opcode.Op), L: yyS[yypt-4].expr, R: variable} } - case 592: - { - parser.yyVAL.item = opcode.GE - } case 593: { - parser.yyVAL.item = opcode.GT + parser.yyVAL.item = opcode.GE } case 594: { - parser.yyVAL.item = opcode.LE + parser.yyVAL.item = opcode.GT } case 595: { - parser.yyVAL.item = opcode.LT + parser.yyVAL.item = opcode.LE } case 596: { - parser.yyVAL.item = opcode.NE + parser.yyVAL.item = opcode.LT } case 597: { @@ -14430,51 +14438,51 @@ yynewstate: } case 598: { - parser.yyVAL.item = opcode.EQ + parser.yyVAL.item = opcode.NE } case 599: { - parser.yyVAL.item = opcode.NullEQ + parser.yyVAL.item = opcode.EQ } case 600: { - parser.yyVAL.item = true + parser.yyVAL.item = opcode.NullEQ } case 601: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 602: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 603: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 604: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 605: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 606: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 607: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 608: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 609: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 610: { @@ -14486,19 +14494,23 @@ yynewstate: } case 612: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 613: { - parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-4].expr, Not: !yyS[yypt-3].item.(bool), List: yyS[yypt-1].item.([]ast.ExprNode)} + parser.yyVAL.item = true } case 614: + { + parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-4].expr, Not: !yyS[yypt-3].item.(bool), List: yyS[yypt-1].item.([]ast.ExprNode)} + } + case 615: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.MultiRows = true parser.yyVAL.expr = &ast.PatternInExpr{Expr: yyS[yypt-2].expr, Not: !yyS[yypt-1].item.(bool), Sel: sq} } - case 615: + case 616: { parser.yyVAL.expr = &ast.BetweenExpr{ Expr: yyS[yypt-4].expr, @@ -14507,7 +14519,7 @@ yynewstate: Not: !yyS[yypt-3].item.(bool), } } - case 616: + case 617: { escape := yyS[yypt-0].ident if len(escape) > 1 { @@ -14523,57 +14535,57 @@ yynewstate: Escape: escape[0], } } - case 617: + case 618: { parser.yyVAL.expr = &ast.PatternRegexpExpr{Expr: yyS[yypt-2].expr, Pattern: yyS[yypt-0].expr, Not: !yyS[yypt-1].item.(bool)} } - case 621: + case 622: { parser.yyVAL.ident = "\\" } - case 622: + case 623: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 623: + case 624: { parser.yyVAL.item = &ast.SelectField{WildCard: &ast.WildCardField{}} } - case 624: + case 625: { wildCard := &ast.WildCardField{Table: model.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } - case 625: + case 626: { wildCard := &ast.WildCardField{Schema: model.NewCIStr(yyS[yypt-4].ident), Table: model.NewCIStr(yyS[yypt-2].ident)} parser.yyVAL.item = &ast.SelectField{WildCard: wildCard} } - case 626: + case 627: { expr := yyS[yypt-1].expr asName := yyS[yypt-0].ident parser.yyVAL.item = &ast.SelectField{Expr: expr, AsName: model.NewCIStr(asName)} } - case 627: + case 628: { parser.yyVAL.ident = "" } - case 630: + case 631: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 632: + case 633: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 633: + case 634: { field := yyS[yypt-0].item.(*ast.SelectField) field.Offset = parser.startOffset(&yyS[yypt]) parser.yyVAL.item = []*ast.SelectField{field} } - case 634: + case 635: { fl := yyS[yypt-2].item.([]*ast.SelectField) last := fl[len(fl)-1] @@ -14585,71 +14597,71 @@ yynewstate: newField.Offset = parser.startOffset(&yyS[yypt]) parser.yyVAL.item = append(fl, newField) } - case 635: + case 636: { parser.yyVAL.item = &ast.GroupByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 636: + case 637: { parser.yyVAL.item = nil } - case 637: + case 638: { parser.yyVAL.item = &ast.HavingClause{Expr: yyS[yypt-0].expr} } - case 638: + case 639: { parser.yyVAL.item = nil } - case 640: + case 641: { parser.yyVAL.item = &ast.AsOfClause{ TsExpr: yyS[yypt-0].expr.(ast.ExprNode), } } - case 641: + case 642: { parser.yyVAL.item = false } - case 642: + case 643: { parser.yyVAL.item = true } - case 643: + case 644: { parser.yyVAL.item = false } - case 644: + case 645: { parser.yyVAL.item = true } - case 645: + case 646: { parser.yyVAL.item = false } - case 646: + case 647: { parser.yyVAL.item = true } - case 647: + case 648: { parser.yyVAL.item = &ast.NullString{ String: "", Empty: false, } } - case 648: + case 649: { parser.yyVAL.item = &ast.NullString{ String: yyS[yypt-0].ident, Empty: len(yyS[yypt-0].ident) == 0, } } - case 649: + case 650: { parser.yyVAL.item = nil } - case 650: + case 651: { // Merge the options if yyS[yypt-1].item == nil { @@ -14673,19 +14685,19 @@ yynewstate: parser.yyVAL.item = opt1 } } - case 651: + case 652: { parser.yyVAL.item = &ast.IndexOption{ KeyBlockSize: yyS[yypt-0].item.(uint64), } } - case 652: + case 653: { parser.yyVAL.item = &ast.IndexOption{ Tp: yyS[yypt-0].item.(model.IndexType), } } - case 653: + case 654: { parser.yyVAL.item = &ast.IndexOption{ ParserName: model.NewCIStr(yyS[yypt-0].ident), @@ -14693,75 +14705,75 @@ yynewstate: yylex.AppendError(yylex.Errorf("The WITH PARASER clause is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 654: + case 655: { parser.yyVAL.item = &ast.IndexOption{ Comment: yyS[yypt-0].ident, } } - case 655: + case 656: { parser.yyVAL.item = &ast.IndexOption{ Visibility: yyS[yypt-0].item.(ast.IndexVisibility), } } - case 656: + case 657: { parser.yyVAL.item = &ast.IndexOption{ PrimaryKeyTp: yyS[yypt-0].item.(model.PrimaryKeyType), } } - case 657: + case 658: { parser.yyVAL.item = []interface{}{yyS[yypt-0].item, nil} } - case 658: + case 659: { parser.yyVAL.item = []interface{}{yyS[yypt-2].item, yyS[yypt-0].item} } - case 659: + case 660: { parser.yyVAL.item = []interface{}{&ast.NullString{String: yyS[yypt-2].ident, Empty: len(yyS[yypt-2].ident) == 0}, yyS[yypt-0].item} } - case 660: + case 661: { parser.yyVAL.item = nil } - case 662: + case 663: { parser.yyVAL.item = yyS[yypt-0].item } - case 663: + case 664: { parser.yyVAL.item = yyS[yypt-0].item } - case 664: + case 665: { parser.yyVAL.item = model.IndexTypeBtree } - case 665: + case 666: { parser.yyVAL.item = model.IndexTypeHash } - case 666: + case 667: { parser.yyVAL.item = model.IndexTypeRtree } - case 667: + case 668: { parser.yyVAL.item = ast.IndexVisibilityVisible } - case 668: + case 669: { parser.yyVAL.item = ast.IndexVisibilityInvisible } - case 1129: + case 1130: { parser.yyVAL.statement = &ast.CallStmt{ Procedure: yyS[yypt-0].expr.(*ast.FuncCallExpr), } } - case 1130: + case 1131: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14769,7 +14781,7 @@ yynewstate: Args: []ast.ExprNode{}, } } - case 1131: + case 1132: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14778,7 +14790,7 @@ yynewstate: Args: []ast.ExprNode{}, } } - case 1132: + case 1133: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14786,7 +14798,7 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1133: + case 1134: { parser.yyVAL.expr = &ast.FuncCallExpr{ Tp: ast.FuncCallExprTypeGeneric, @@ -14795,7 +14807,7 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1134: + case 1135: { x := yyS[yypt-1].item.(*ast.InsertStmt) x.Priority = yyS[yypt-6].item.(mysql.PriorityEnum) @@ -14812,17 +14824,13 @@ yynewstate: x.PartitionNames = yyS[yypt-2].item.([]model.CIStr) parser.yyVAL.statement = x } - case 1137: + case 1138: { parser.yyVAL.item = &ast.InsertStmt{ Columns: yyS[yypt-3].item.([]*ast.ColumnName), Lists: yyS[yypt-0].item.([][]ast.ExprNode), } } - case 1138: - { - parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} - } case 1139: { parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} @@ -14832,6 +14840,10 @@ yynewstate: parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} } case 1141: + { + parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + } + case 1142: { var sel ast.ResultSetNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -14844,13 +14856,9 @@ yynewstate: } parser.yyVAL.item = &ast.InsertStmt{Columns: yyS[yypt-2].item.([]*ast.ColumnName), Select: sel} } - case 1142: - { - parser.yyVAL.item = &ast.InsertStmt{Lists: yyS[yypt-0].item.([][]ast.ExprNode)} - } case 1143: { - parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + parser.yyVAL.item = &ast.InsertStmt{Lists: yyS[yypt-0].item.([][]ast.ExprNode)} } case 1144: { @@ -14861,6 +14869,10 @@ yynewstate: parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} } case 1146: + { + parser.yyVAL.item = &ast.InsertStmt{Select: yyS[yypt-0].statement.(ast.ResultSetNode)} + } + case 1147: { var sel ast.ResultSetNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -14873,66 +14885,66 @@ yynewstate: } parser.yyVAL.item = &ast.InsertStmt{Select: sel} } - case 1147: + case 1148: { parser.yyVAL.item = &ast.InsertStmt{Setlist: yyS[yypt-0].item.([]*ast.Assignment)} } - case 1150: + case 1151: { parser.yyVAL.item = [][]ast.ExprNode{yyS[yypt-0].item.([]ast.ExprNode)} } - case 1151: + case 1152: { parser.yyVAL.item = append(yyS[yypt-2].item.([][]ast.ExprNode), yyS[yypt-0].item.([]ast.ExprNode)) } - case 1152: + case 1153: { parser.yyVAL.item = yyS[yypt-1].item } - case 1153: + case 1154: { parser.yyVAL.item = []ast.ExprNode{} } - case 1155: + case 1156: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 1156: + case 1157: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 1158: + case 1159: { parser.yyVAL.expr = &ast.DefaultExpr{} } - case 1159: + case 1160: { parser.yyVAL.item = &ast.Assignment{ Column: yyS[yypt-2].item.(*ast.ColumnName), Expr: yyS[yypt-0].expr, } } - case 1160: + case 1161: { parser.yyVAL.item = []*ast.Assignment{} } - case 1161: + case 1162: { parser.yyVAL.item = []*ast.Assignment{yyS[yypt-0].item.(*ast.Assignment)} } - case 1162: + case 1163: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.Assignment), yyS[yypt-0].item.(*ast.Assignment)) } - case 1163: + case 1164: { parser.yyVAL.item = nil } - case 1164: + case 1165: { parser.yyVAL.item = yyS[yypt-0].item } - case 1165: + case 1166: { x := yyS[yypt-0].item.(*ast.InsertStmt) x.IsReplace = true @@ -14942,31 +14954,31 @@ yynewstate: x.PartitionNames = yyS[yypt-1].item.([]model.CIStr) parser.yyVAL.statement = x } - case 1166: + case 1167: { parser.yyVAL.expr = ast.NewValueExpr(false, parser.charset, parser.collation) } - case 1167: + case 1168: { parser.yyVAL.expr = ast.NewValueExpr(nil, parser.charset, parser.collation) } - case 1168: + case 1169: { parser.yyVAL.expr = ast.NewValueExpr(true, parser.charset, parser.collation) } - case 1169: + case 1170: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1170: + case 1171: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1171: + case 1172: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1173: + case 1174: { // See https://dev.mysql.com/doc/refman/5.7/en/charset-literal.html co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) @@ -14983,15 +14995,15 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1174: + case 1175: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1175: + case 1176: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1176: + case 1177: { co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) if err != nil { @@ -15007,7 +15019,7 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1177: + case 1178: { co, err := charset.GetDefaultCollationLegacy(yyS[yypt-1].ident) if err != nil { @@ -15023,12 +15035,12 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1178: + case 1179: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) parser.yyVAL.expr = expr } - case 1179: + case 1180: { valExpr := yyS[yypt-1].expr.(ast.ValueExpr) strLit := valExpr.GetString() @@ -15041,31 +15053,31 @@ yynewstate: } parser.yyVAL.expr = expr } - case 1180: + case 1181: { parser.yyVAL.item = []*ast.AlterOrderItem{yyS[yypt-0].item.(*ast.AlterOrderItem)} } - case 1181: + case 1182: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.AlterOrderItem), yyS[yypt-0].item.(*ast.AlterOrderItem)) } - case 1182: + case 1183: { parser.yyVAL.item = &ast.AlterOrderItem{Column: yyS[yypt-1].item.(*ast.ColumnName), Desc: yyS[yypt-0].item.(bool)} } - case 1183: + case 1184: { parser.yyVAL.item = &ast.OrderByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1184: + case 1185: { parser.yyVAL.item = []*ast.ByItem{yyS[yypt-0].item.(*ast.ByItem)} } - case 1185: + case 1186: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.ByItem), yyS[yypt-0].item.(*ast.ByItem)) } - case 1186: + case 1187: { expr := yyS[yypt-0].expr valueExpr, ok := expr.(ast.ValueExpr) @@ -15077,7 +15089,7 @@ yynewstate: } parser.yyVAL.item = &ast.ByItem{Expr: expr, NullOrder: true} } - case 1187: + case 1188: { expr := yyS[yypt-1].expr valueExpr, ok := expr.(ast.ValueExpr) @@ -15089,55 +15101,55 @@ yynewstate: } parser.yyVAL.item = &ast.ByItem{Expr: expr, Desc: yyS[yypt-0].item.(bool)} } - case 1188: + case 1189: { parser.yyVAL.item = false } - case 1189: + case 1190: { parser.yyVAL.item = true } - case 1190: + case 1191: { parser.yyVAL.item = false // ASC by default } - case 1191: + case 1192: { parser.yyVAL.item = false } - case 1192: + case 1193: { parser.yyVAL.item = true } - case 1193: + case 1194: { parser.yyVAL.item = nil } - case 1195: + case 1196: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Or, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1196: + case 1197: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.And, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1197: + case 1198: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.LeftShift, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1198: + case 1199: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.RightShift, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1199: + case 1200: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Plus, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1200: + case 1201: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Minus, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1201: + case 1202: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr("DATE_ADD"), @@ -15148,7 +15160,7 @@ yynewstate: }, } } - case 1202: + case 1203: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr("DATE_SUB"), @@ -15159,44 +15171,44 @@ yynewstate: }, } } - case 1203: + case 1204: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mul, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1204: + case 1205: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Div, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1205: + case 1206: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1206: + case 1207: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.IntDiv, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1207: + case 1208: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1208: + case 1209: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Xor, L: yyS[yypt-2].expr, R: yyS[yypt-0].expr} } - case 1210: + case 1211: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1211: + case 1212: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Table: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1212: + case 1213: { parser.yyVAL.expr = &ast.ColumnNameExpr{Name: &ast.ColumnName{ Schema: model.NewCIStr(yyS[yypt-4].ident), @@ -15204,39 +15216,39 @@ yynewstate: Name: model.NewCIStr(yyS[yypt-0].ident), }} } - case 1217: + case 1218: { parser.yyVAL.expr = &ast.SetCollationExpr{Expr: yyS[yypt-2].expr, Collate: yyS[yypt-0].ident} } - case 1220: + case 1221: { parser.yyVAL.expr = ast.NewParamMarkerExpr(yyS[yypt].offset) } - case 1223: + case 1224: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not2, V: yyS[yypt-0].expr} } - case 1224: + case 1225: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.BitNeg, V: yyS[yypt-0].expr} } - case 1225: + case 1226: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Minus, V: yyS[yypt-0].expr} } - case 1226: + case 1227: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Plus, V: yyS[yypt-0].expr} } - case 1227: + case 1228: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.Concat), Args: []ast.ExprNode{yyS[yypt-2].expr, yyS[yypt-0].expr}} } - case 1228: + case 1229: { parser.yyVAL.expr = &ast.UnaryOperationExpr{Op: opcode.Not2, V: yyS[yypt-0].expr} } - case 1230: + case 1231: { startOffset := parser.startOffset(&yyS[yypt-1]) endOffset := parser.endOffset(&yyS[yypt]) @@ -15244,23 +15256,23 @@ yynewstate: expr.SetText(parser.lexer.client, parser.src[startOffset:endOffset]) parser.yyVAL.expr = &ast.ParenthesesExpr{Expr: expr} } - case 1231: + case 1232: { values := append(yyS[yypt-3].item.([]ast.ExprNode), yyS[yypt-1].expr) parser.yyVAL.expr = &ast.RowExpr{Values: values} } - case 1232: + case 1233: { values := append(yyS[yypt-3].item.([]ast.ExprNode), yyS[yypt-1].expr) parser.yyVAL.expr = &ast.RowExpr{Values: values} } - case 1233: + case 1234: { sq := yyS[yypt-0].expr.(*ast.SubqueryExpr) sq.Exists = true parser.yyVAL.expr = &ast.ExistsSubqueryExpr{Sel: sq} } - case 1234: + case 1235: { /* * ODBC escape syntax. @@ -15284,7 +15296,7 @@ yynewstate: parser.yyVAL.expr = yyS[yypt-1].expr } } - case 1235: + case 1236: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#operator_binary x := types.NewFieldType(mysql.TypeString) @@ -15297,7 +15309,7 @@ yynewstate: FunctionType: ast.CastBinaryOperator, } } - case 1236: + case 1237: { /* See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_cast */ tp := yyS[yypt-1].item.(*types.FieldType) @@ -15317,7 +15329,7 @@ yynewstate: ExplicitCharSet: explicitCharset, } } - case 1237: + case 1238: { x := &ast.CaseExpr{WhenClauses: yyS[yypt-2].item.([]*ast.WhenClause)} if yyS[yypt-3].expr != nil { @@ -15328,7 +15340,7 @@ yynewstate: } parser.yyVAL.expr = x } - case 1238: + case 1239: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert tp := yyS[yypt-1].item.(*types.FieldType) @@ -15348,7 +15360,7 @@ yynewstate: ExplicitCharSet: explicitCharset, } } - case 1239: + case 1240: { // See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") @@ -15357,62 +15369,62 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-3].expr, charset1}, } } - case 1240: + case 1241: { parser.yyVAL.expr = &ast.DefaultExpr{Name: yyS[yypt-1].expr.(*ast.ColumnNameExpr).Name} } - case 1241: + case 1242: { parser.yyVAL.expr = &ast.ValuesExpr{Column: yyS[yypt-1].expr.(*ast.ColumnNameExpr)} } - case 1242: + case 1243: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} } - case 1243: + case 1244: { expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) extract := &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONExtract), Args: []ast.ExprNode{yyS[yypt-2].expr, expr}} parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.JSONUnquote), Args: []ast.ExprNode{extract}} } - case 1246: + case 1247: { parser.yyVAL.item = false } - case 1247: + case 1248: { parser.yyVAL.item = true } - case 1248: + case 1249: { parser.yyVAL.item = false } - case 1250: + case 1251: { parser.yyVAL.item = true } - case 1253: + case 1254: { parser.yyVAL.item = true } - case 1295: + case 1296: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1296: + case 1297: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1297: + case 1298: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident)} } - case 1298: + case 1299: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-2].ident)} } - case 1299: + case 1300: { args := []ast.ExprNode{} if yyS[yypt-0].item != nil { @@ -15420,7 +15432,7 @@ yynewstate: } parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-1].ident), Args: args} } - case 1300: + case 1301: { nilVal := ast.NewValueExpr(nil, parser.charset, parser.collation) args := yyS[yypt-1].item.([]ast.ExprNode) @@ -15429,7 +15441,7 @@ yynewstate: Args: append(args, nilVal), } } - case 1301: + case 1302: { charset1 := ast.NewValueExpr(yyS[yypt-1].ident, "", "") args := yyS[yypt-3].item.([]ast.ExprNode) @@ -15438,42 +15450,42 @@ yynewstate: Args: append(args, charset1), } } - case 1302: + case 1303: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.DateLiteral), Args: []ast.ExprNode{expr}} } - case 1303: + case 1304: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimeLiteral), Args: []ast.ExprNode{expr}} } - case 1304: + case 1305: { expr := ast.NewValueExpr(yyS[yypt-0].ident, "", "") parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.TimestampLiteral), Args: []ast.ExprNode{expr}} } - case 1305: + case 1306: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.InsertFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1306: + case 1307: { parser.yyVAL.expr = &ast.BinaryOperationExpr{Op: opcode.Mod, L: yyS[yypt-3].expr, R: yyS[yypt-1].expr} } - case 1307: + case 1308: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(ast.PasswordFunc), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1308: + case 1309: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1309: + case 1310: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1310: + case 1311: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), @@ -15484,7 +15496,7 @@ yynewstate: }, } } - case 1311: + case 1312: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), @@ -15495,7 +15507,7 @@ yynewstate: }, } } - case 1312: + case 1313: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), @@ -15506,7 +15518,7 @@ yynewstate: }, } } - case 1313: + case 1314: { timeUnit := &ast.TimeUnitExpr{Unit: yyS[yypt-3].item.(ast.TimeUnitType)} parser.yyVAL.expr = &ast.FuncCallExpr{ @@ -15514,7 +15526,7 @@ yynewstate: Args: []ast.ExprNode{timeUnit, yyS[yypt-1].expr}, } } - case 1314: + case 1315: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), @@ -15524,67 +15536,67 @@ yynewstate: }, } } - case 1315: + case 1316: { parser.yyVAL.expr = &ast.FuncCallExpr{FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}} } - case 1316: + case 1317: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1317: + case 1318: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1318: + case 1319: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1319: + case 1320: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1320: + case 1321: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1321: + case 1322: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{&ast.TimeUnitExpr{Unit: yyS[yypt-5].item.(ast.TimeUnitType)}, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1322: + case 1323: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } - case 1323: + case 1324: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-5].ident), Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr}, } } - case 1324: + case 1325: { spaceVal := ast.NewValueExpr(" ", parser.charset, parser.collation) direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-3].item.(ast.TrimDirectionType)} @@ -15593,7 +15605,7 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-1].expr, spaceVal, direction}, } } - case 1325: + case 1326: { direction := &ast.TrimDirectionExpr{Direction: yyS[yypt-4].item.(ast.TrimDirectionType)} parser.yyVAL.expr = &ast.FuncCallExpr{ @@ -15601,63 +15613,63 @@ yynewstate: Args: []ast.ExprNode{yyS[yypt-1].expr, yyS[yypt-3].expr, direction}, } } - case 1326: + case 1327: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: []ast.ExprNode{yyS[yypt-1].expr}, } } - case 1327: + case 1328: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("CHAR", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } - case 1328: + case 1329: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-6].ident), Args: []ast.ExprNode{yyS[yypt-4].expr, ast.NewValueExpr("BINARY", parser.charset, parser.collation), ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)}, } } - case 1330: + case 1331: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-7].ident), Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-3].expr, yyS[yypt-1].expr}, } } - case 1331: + case 1332: { parser.yyVAL.item = ast.GetFormatSelectorDate } - case 1332: + case 1333: { parser.yyVAL.item = ast.GetFormatSelectorDatetime } - case 1333: + case 1334: { parser.yyVAL.item = ast.GetFormatSelectorTime } - case 1334: + case 1335: { parser.yyVAL.item = ast.GetFormatSelectorDatetime } - case 1339: + case 1340: { parser.yyVAL.item = ast.TrimBoth } - case 1340: + case 1341: { parser.yyVAL.item = ast.TrimLeading } - case 1341: + case 1342: { parser.yyVAL.item = ast.TrimTrailing } - case 1342: + case 1343: { objNameExpr := &ast.TableNameExpr{ Name: yyS[yypt-1].item.(*ast.TableName), @@ -15667,7 +15679,7 @@ yynewstate: Args: []ast.ExprNode{objNameExpr}, } } - case 1343: + case 1344: { objNameExpr := &ast.TableNameExpr{ Name: yyS[yypt-3].item.(*ast.TableName), @@ -15678,7 +15690,7 @@ yynewstate: Args: []ast.ExprNode{objNameExpr, valueExpr}, } } - case 1345: + case 1346: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15686,15 +15698,15 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1346: + case 1347: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-3].ident, Args: yyS[yypt-1].item.([]ast.ExprNode), Distinct: false} } - case 1347: + case 1348: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-3].ident, Args: yyS[yypt-1].item.([]ast.ExprNode)} } - case 1348: + case 1349: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15702,7 +15714,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1349: + case 1350: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15710,7 +15722,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1350: + case 1351: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15718,7 +15730,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1351: + case 1352: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15726,7 +15738,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1352: + case 1353: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15734,7 +15746,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1353: + case 1354: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15742,11 +15754,11 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1354: + case 1355: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: yyS[yypt-1].item.([]ast.ExprNode), Distinct: true} } - case 1355: + case 1356: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15754,7 +15766,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1356: + case 1357: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15762,7 +15774,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1357: + case 1358: { args := []ast.ExprNode{ast.NewValueExpr(1, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -15771,7 +15783,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: args} } } - case 1358: + case 1359: { args := yyS[yypt-4].item.([]ast.ExprNode) args = append(args, yyS[yypt-2].item.(ast.ExprNode)) @@ -15785,7 +15797,7 @@ yynewstate: parser.yyVAL.expr = agg } } - case 1359: + case 1360: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15793,7 +15805,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1360: + case 1361: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15801,7 +15813,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1361: + case 1362: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15809,7 +15821,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1362: + case 1363: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: ast.AggFuncStddevPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15817,7 +15829,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: ast.AggFuncStddevPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1363: + case 1364: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15825,7 +15837,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1364: + case 1365: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: ast.AggFuncVarPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool), Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15833,11 +15845,11 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: ast.AggFuncVarPop, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } } - case 1365: + case 1366: { parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Distinct: yyS[yypt-3].item.(bool)} } - case 1366: + case 1367: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15845,7 +15857,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1367: + case 1368: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15853,7 +15865,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}} } } - case 1368: + case 1369: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15861,7 +15873,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-6].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}} } } - case 1369: + case 1370: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15869,7 +15881,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-4].expr, yyS[yypt-2].expr}} } } - case 1370: + case 1371: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15877,7 +15889,7 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-7].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}} } } - case 1371: + case 1372: { if yyS[yypt-0].item != nil { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}, Spec: *(yyS[yypt-0].item.(*ast.WindowSpec))} @@ -15885,22 +15897,22 @@ yynewstate: parser.yyVAL.expr = &ast.AggregateFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-5].expr, yyS[yypt-2].expr}} } } - case 1372: + case 1373: { parser.yyVAL.item = ast.NewValueExpr(",", "", "") } - case 1373: + case 1374: { parser.yyVAL.item = ast.NewValueExpr(yyS[yypt-0].ident, "", "") } - case 1374: + case 1375: { parser.yyVAL.expr = &ast.FuncCallExpr{ FnName: model.NewCIStr(yyS[yypt-3].ident), Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1375: + case 1376: { var tp ast.FuncCallExprType if isInTokenMap(yyS[yypt-3].ident) { @@ -15915,159 +15927,159 @@ yynewstate: Args: yyS[yypt-1].item.([]ast.ExprNode), } } - case 1376: + case 1377: { parser.yyVAL.item = nil } - case 1377: + case 1378: { parser.yyVAL.item = nil } - case 1378: + case 1379: { expr := ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation) parser.yyVAL.item = expr } - case 1380: + case 1381: { parser.yyVAL.item = ast.TimeUnitSecondMicrosecond } - case 1381: + case 1382: { parser.yyVAL.item = ast.TimeUnitMinuteMicrosecond } - case 1382: + case 1383: { parser.yyVAL.item = ast.TimeUnitMinuteSecond } - case 1383: + case 1384: { parser.yyVAL.item = ast.TimeUnitHourMicrosecond } - case 1384: + case 1385: { parser.yyVAL.item = ast.TimeUnitHourSecond } - case 1385: + case 1386: { parser.yyVAL.item = ast.TimeUnitHourMinute } - case 1386: + case 1387: { parser.yyVAL.item = ast.TimeUnitDayMicrosecond } - case 1387: + case 1388: { parser.yyVAL.item = ast.TimeUnitDaySecond } - case 1388: + case 1389: { parser.yyVAL.item = ast.TimeUnitDayMinute } - case 1389: + case 1390: { parser.yyVAL.item = ast.TimeUnitDayHour } - case 1390: + case 1391: { parser.yyVAL.item = ast.TimeUnitYearMonth } - case 1391: + case 1392: { parser.yyVAL.item = ast.TimeUnitMicrosecond } - case 1392: + case 1393: { parser.yyVAL.item = ast.TimeUnitSecond } - case 1393: + case 1394: { parser.yyVAL.item = ast.TimeUnitMinute } - case 1394: + case 1395: { parser.yyVAL.item = ast.TimeUnitHour } - case 1395: + case 1396: { parser.yyVAL.item = ast.TimeUnitDay } - case 1396: + case 1397: { parser.yyVAL.item = ast.TimeUnitWeek } - case 1397: + case 1398: { parser.yyVAL.item = ast.TimeUnitMonth } - case 1398: + case 1399: { parser.yyVAL.item = ast.TimeUnitQuarter } - case 1399: + case 1400: { parser.yyVAL.item = ast.TimeUnitYear } - case 1400: + case 1401: { parser.yyVAL.item = ast.TimeUnitSecond } - case 1401: + case 1402: { parser.yyVAL.item = ast.TimeUnitMinute } - case 1402: + case 1403: { parser.yyVAL.item = ast.TimeUnitHour } - case 1403: + case 1404: { parser.yyVAL.item = ast.TimeUnitDay } - case 1404: + case 1405: { parser.yyVAL.item = ast.TimeUnitWeek } - case 1405: + case 1406: { parser.yyVAL.item = ast.TimeUnitMonth } - case 1406: + case 1407: { parser.yyVAL.item = ast.TimeUnitQuarter } - case 1407: + case 1408: { parser.yyVAL.item = ast.TimeUnitYear } - case 1408: + case 1409: { parser.yyVAL.expr = nil } - case 1410: + case 1411: { parser.yyVAL.item = []*ast.WhenClause{yyS[yypt-0].item.(*ast.WhenClause)} } - case 1411: + case 1412: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.WhenClause), yyS[yypt-0].item.(*ast.WhenClause)) } - case 1412: + case 1413: { parser.yyVAL.item = &ast.WhenClause{ Expr: yyS[yypt-2].expr, Result: yyS[yypt-0].expr, } } - case 1413: + case 1414: { parser.yyVAL.item = nil } - case 1414: + case 1415: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1415: + case 1416: { x := types.NewFieldType(mysql.TypeVarString) x.Flen = yyS[yypt-0].item.(int) // TODO: Flen should be the flen of expression @@ -16079,7 +16091,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1416: + case 1417: { x := types.NewFieldType(mysql.TypeVarString) x.Flen = yyS[yypt-1].item.(int) // TODO: Flen should be the flen of expression @@ -16102,7 +16114,7 @@ yynewstate: } parser.yyVAL.item = x } - case 1417: + case 1418: { x := types.NewFieldType(mysql.TypeDate) x.Charset = charset.CharsetBin @@ -16110,7 +16122,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1418: + case 1419: { x := types.NewFieldType(mysql.TypeYear) x.Charset = charset.CharsetBin @@ -16118,7 +16130,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1419: + case 1420: { x := types.NewFieldType(mysql.TypeDatetime) x.Flen, _ = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDatetime) @@ -16131,7 +16143,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1420: + case 1421: { fopt := yyS[yypt-0].item.(*ast.FloatOpt) x := types.NewFieldType(mysql.TypeNewDecimal) @@ -16142,7 +16154,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1421: + case 1422: { x := types.NewFieldType(mysql.TypeDuration) x.Flen, _ = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDuration) @@ -16155,7 +16167,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1422: + case 1423: { x := types.NewFieldType(mysql.TypeLonglong) x.Charset = charset.CharsetBin @@ -16163,7 +16175,7 @@ yynewstate: x.Flag |= mysql.BinaryFlag parser.yyVAL.item = x } - case 1423: + case 1424: { x := types.NewFieldType(mysql.TypeLonglong) x.Flag |= mysql.UnsignedFlag | mysql.BinaryFlag @@ -16171,7 +16183,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1424: + case 1425: { x := types.NewFieldType(mysql.TypeJSON) x.Flag |= mysql.BinaryFlag | (mysql.ParseToJSONFlag) @@ -16179,7 +16191,7 @@ yynewstate: x.Collate = mysql.DefaultCollationName parser.yyVAL.item = x } - case 1425: + case 1426: { x := types.NewFieldType(mysql.TypeDouble) x.Flen, x.Decimal = mysql.GetDefaultFieldLengthAndDecimalForCast(mysql.TypeDouble) @@ -16188,7 +16200,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1426: + case 1427: { x := types.NewFieldType(mysql.TypeFloat) fopt := yyS[yypt-0].item.(*ast.FloatOpt) @@ -16203,7 +16215,7 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1427: + case 1428: { var x *types.FieldType if parser.lexer.GetSQLMode().HasRealAsFloatMode() { @@ -16217,65 +16229,65 @@ yynewstate: x.Collate = charset.CollationBin parser.yyVAL.item = x } - case 1428: + case 1429: { parser.yyVAL.item = mysql.LowPriority } - case 1429: + case 1430: { parser.yyVAL.item = mysql.HighPriority } - case 1430: + case 1431: { parser.yyVAL.item = mysql.DelayedPriority } - case 1431: + case 1432: { parser.yyVAL.item = mysql.NoPriority } - case 1433: + case 1434: { parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-0].ident)} } - case 1434: + case 1435: { parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(yyS[yypt-2].ident), Name: model.NewCIStr(yyS[yypt-0].ident)} } - case 1435: + case 1436: { tbl := []*ast.TableName{yyS[yypt-0].item.(*ast.TableName)} parser.yyVAL.item = tbl } - case 1436: + case 1437: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.TableName), yyS[yypt-0].item.(*ast.TableName)) } - case 1437: + case 1438: { parser.yyVAL.item = &ast.TableName{Name: model.NewCIStr(yyS[yypt-1].ident)} } - case 1438: + case 1439: { parser.yyVAL.item = &ast.TableName{Schema: model.NewCIStr(yyS[yypt-3].ident), Name: model.NewCIStr(yyS[yypt-1].ident)} } - case 1439: + case 1440: { tbl := []*ast.TableName{yyS[yypt-0].item.(*ast.TableName)} parser.yyVAL.item = tbl } - case 1440: + case 1441: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.TableName), yyS[yypt-0].item.(*ast.TableName)) } - case 1443: + case 1444: { parser.yyVAL.item = false } - case 1444: + case 1445: { parser.yyVAL.item = true } - case 1445: + case 1446: { var sqlText string var sqlVar *ast.VariableExpr @@ -16291,86 +16303,86 @@ yynewstate: SQLVar: sqlVar, } } - case 1446: + case 1447: { parser.yyVAL.item = yyS[yypt-0].ident } - case 1447: + case 1448: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1448: + case 1449: { parser.yyVAL.statement = &ast.ExecuteStmt{Name: yyS[yypt-0].ident} } - case 1449: + case 1450: { parser.yyVAL.statement = &ast.ExecuteStmt{ Name: yyS[yypt-2].ident, UsingVars: yyS[yypt-0].item.([]ast.ExprNode), } } - case 1450: + case 1451: { parser.yyVAL.item = []ast.ExprNode{yyS[yypt-0].expr} } - case 1451: + case 1452: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.ExprNode), yyS[yypt-0].expr) } - case 1452: + case 1453: { parser.yyVAL.statement = &ast.DeallocateStmt{Name: yyS[yypt-0].ident} } - case 1455: + case 1456: { parser.yyVAL.statement = &ast.RollbackStmt{} } - case 1456: + case 1457: { parser.yyVAL.statement = &ast.RollbackStmt{CompletionType: yyS[yypt-0].item.(ast.CompletionType)} } - case 1457: + case 1458: { parser.yyVAL.item = ast.CompletionTypeChain } - case 1458: + case 1459: { parser.yyVAL.item = ast.CompletionTypeRelease } - case 1459: + case 1460: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1460: + case 1461: { parser.yyVAL.item = ast.CompletionTypeChain } - case 1461: + case 1462: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1462: + case 1463: { parser.yyVAL.item = ast.CompletionTypeRelease } - case 1463: + case 1464: { parser.yyVAL.item = ast.CompletionTypeDefault } - case 1464: + case 1465: { parser.yyVAL.statement = &ast.ShutdownStmt{} } - case 1465: + case 1466: { parser.yyVAL.statement = &ast.RestartStmt{} } - case 1466: + case 1467: { parser.yyVAL.statement = &ast.HelpStmt{Topic: yyS[yypt-0].ident} } - case 1467: + case 1468: { st := &ast.SelectStmt{ SelectStmtOpts: yyS[yypt-1].item.(*ast.SelectStmtOpts), @@ -16383,7 +16395,7 @@ yynewstate: } parser.yyVAL.item = st } - case 1468: + case 1469: { st := yyS[yypt-2].item.(*ast.SelectStmt) lastField := st.Fields.Fields[len(st.Fields.Fields)-1] @@ -16395,7 +16407,7 @@ yynewstate: st.Where = yyS[yypt-0].item.(ast.ExprNode) } } - case 1469: + case 1470: { st := yyS[yypt-6].item.(*ast.SelectStmt) st.From = yyS[yypt-4].item.(*ast.TableRefsClause) @@ -16418,11 +16430,11 @@ yynewstate: } parser.yyVAL.item = st } - case 1470: + case 1471: { parser.yyVAL.item = nil } - case 1471: + case 1472: { var repSeed ast.ExprNode if yyS[yypt-0].expr != nil { @@ -16435,7 +16447,7 @@ yynewstate: RepeatableSeed: repSeed, } } - case 1472: + case 1473: { var repSeed ast.ExprNode if yyS[yypt-0].expr != nil { @@ -16446,43 +16458,43 @@ yynewstate: RepeatableSeed: repSeed, } } - case 1473: + case 1474: { parser.yyVAL.item = ast.SampleMethodTypeNone } - case 1474: + case 1475: { parser.yyVAL.item = ast.SampleMethodTypeSystem } - case 1475: + case 1476: { parser.yyVAL.item = ast.SampleMethodTypeBernoulli } - case 1476: + case 1477: { parser.yyVAL.item = ast.SampleMethodTypeTiDBRegion } - case 1477: + case 1478: { parser.yyVAL.item = ast.SampleClauseUnitTypeDefault } - case 1478: + case 1479: { parser.yyVAL.item = ast.SampleClauseUnitTypeRow } - case 1479: + case 1480: { parser.yyVAL.item = ast.SampleClauseUnitTypePercent } - case 1480: + case 1481: { parser.yyVAL.expr = nil } - case 1481: + case 1482: { parser.yyVAL.expr = yyS[yypt-1].expr } - case 1482: + case 1483: { st := yyS[yypt-6].item.(*ast.SelectStmt) if yyS[yypt-1].item != nil { @@ -16529,7 +16541,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1483: + case 1484: { st := yyS[yypt-5].item.(*ast.SelectStmt) if yyS[yypt-4].item != nil { @@ -16549,7 +16561,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1484: + case 1485: { st := yyS[yypt-4].item.(*ast.SelectStmt) if yyS[yypt-1].item != nil { @@ -16566,7 +16578,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1485: + case 1486: { st := &ast.SelectStmt{ Kind: ast.SelectStmtKindTable, @@ -16588,7 +16600,7 @@ yynewstate: } parser.yyVAL.statement = st } - case 1486: + case 1487: { st := &ast.SelectStmt{ Kind: ast.SelectStmtKindValues, @@ -16609,13 +16621,13 @@ yynewstate: } parser.yyVAL.statement = st } - case 1487: + case 1488: { sel := yyS[yypt-0].statement.(*ast.SelectStmt) sel.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = sel } - case 1488: + case 1489: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -16631,30 +16643,30 @@ yynewstate: } parser.yyVAL.statement = sel } - case 1489: + case 1490: { parser.yyVAL.item = yyS[yypt-0].item } - case 1490: + case 1491: { ws := yyS[yypt-0].item.(*ast.WithClause) ws.IsRecursive = true parser.yyVAL.item = ws } - case 1491: + case 1492: { ws := yyS[yypt-2].item.(*ast.WithClause) ws.CTEs = append(ws.CTEs, yyS[yypt-0].item.(*ast.CommonTableExpression)) parser.yyVAL.item = ws } - case 1492: + case 1493: { ws := &ast.WithClause{} ws.CTEs = make([]*ast.CommonTableExpression, 0, 4) ws.CTEs = append(ws.CTEs, yyS[yypt-0].item.(*ast.CommonTableExpression)) parser.yyVAL.item = ws } - case 1493: + case 1494: { cte := &ast.CommonTableExpression{} cte.Name = model.NewCIStr(yyS[yypt-3].ident) @@ -16662,37 +16674,37 @@ yynewstate: cte.Query = yyS[yypt-0].expr.(*ast.SubqueryExpr) parser.yyVAL.item = cte } - case 1495: + case 1496: { parser.yyVAL.item = nil } - case 1496: + case 1497: { parser.yyVAL.item = yyS[yypt-0].item.([]ast.WindowSpec) } - case 1497: + case 1498: { parser.yyVAL.item = []ast.WindowSpec{yyS[yypt-0].item.(ast.WindowSpec)} } - case 1498: + case 1499: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.WindowSpec), yyS[yypt-0].item.(ast.WindowSpec)) } - case 1499: + case 1500: { var spec = yyS[yypt-0].item.(ast.WindowSpec) spec.Name = yyS[yypt-2].item.(model.CIStr) parser.yyVAL.item = spec } - case 1500: + case 1501: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1501: + case 1502: { parser.yyVAL.item = yyS[yypt-1].item.(ast.WindowSpec) } - case 1502: + case 1503: { spec := ast.WindowSpec{Ref: yyS[yypt-3].item.(model.CIStr)} if yyS[yypt-2].item != nil { @@ -16706,117 +16718,113 @@ yynewstate: } parser.yyVAL.item = spec } - case 1503: + case 1504: { parser.yyVAL.item = model.CIStr{} } - case 1505: + case 1506: { parser.yyVAL.item = nil } - case 1506: + case 1507: { parser.yyVAL.item = &ast.PartitionByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1507: + case 1508: { parser.yyVAL.item = nil } - case 1508: + case 1509: { parser.yyVAL.item = &ast.OrderByClause{Items: yyS[yypt-0].item.([]*ast.ByItem)} } - case 1509: + case 1510: { parser.yyVAL.item = nil } - case 1510: + case 1511: { parser.yyVAL.item = &ast.FrameClause{ Type: yyS[yypt-1].item.(ast.FrameType), Extent: yyS[yypt-0].item.(ast.FrameExtent), } } - case 1511: + case 1512: { parser.yyVAL.item = ast.FrameType(ast.Rows) } - case 1512: + case 1513: { parser.yyVAL.item = ast.FrameType(ast.Ranges) } - case 1513: + case 1514: { parser.yyVAL.item = ast.FrameType(ast.Groups) } - case 1514: + case 1515: { parser.yyVAL.item = ast.FrameExtent{ Start: yyS[yypt-0].item.(ast.FrameBound), End: ast.FrameBound{Type: ast.CurrentRow}, } } - case 1516: + case 1517: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, UnBounded: true} } - case 1517: + case 1518: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} } - case 1518: + case 1519: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: ast.NewParamMarkerExpr(yyS[yypt].offset)} } - case 1519: + case 1520: { parser.yyVAL.item = ast.FrameBound{Type: ast.Preceding, Expr: yyS[yypt-2].expr, Unit: yyS[yypt-1].item.(ast.TimeUnitType)} } - case 1520: + case 1521: { parser.yyVAL.item = ast.FrameBound{Type: ast.CurrentRow} } - case 1521: + case 1522: { parser.yyVAL.item = ast.FrameExtent{Start: yyS[yypt-2].item.(ast.FrameBound), End: yyS[yypt-0].item.(ast.FrameBound)} } - case 1523: + case 1524: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, UnBounded: true} } - case 1524: + case 1525: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} } - case 1525: + case 1526: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: ast.NewParamMarkerExpr(yyS[yypt].offset)} } - case 1526: + case 1527: { parser.yyVAL.item = ast.FrameBound{Type: ast.Following, Expr: yyS[yypt-2].expr, Unit: yyS[yypt-1].item.(ast.TimeUnitType)} } - case 1527: + case 1528: { parser.yyVAL.item = nil } - case 1528: + case 1529: { spec := yyS[yypt-0].item.(ast.WindowSpec) parser.yyVAL.item = &spec } - case 1529: + case 1530: { parser.yyVAL.item = yyS[yypt-0].item.(ast.WindowSpec) } - case 1530: + case 1531: { parser.yyVAL.item = ast.WindowSpec{Name: yyS[yypt-0].item.(model.CIStr), OnlyAlias: true} } - case 1532: - { - parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-3].ident, Spec: yyS[yypt-0].item.(ast.WindowSpec)} - } case 1533: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-3].ident, Spec: yyS[yypt-0].item.(ast.WindowSpec)} @@ -16835,9 +16843,13 @@ yynewstate: } case 1537: { - parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: yyS[yypt-0].item.(ast.WindowSpec)} + parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-3].ident, Spec: yyS[yypt-0].item.(ast.WindowSpec)} } case 1538: + { + parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-4].ident, Args: []ast.ExprNode{yyS[yypt-2].expr}, Spec: yyS[yypt-0].item.(ast.WindowSpec)} + } + case 1539: { args := []ast.ExprNode{yyS[yypt-4].expr} if yyS[yypt-3].item != nil { @@ -16845,7 +16857,7 @@ yynewstate: } parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: args, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1539: + case 1540: { args := []ast.ExprNode{yyS[yypt-4].expr} if yyS[yypt-3].item != nil { @@ -16853,23 +16865,23 @@ yynewstate: } parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-6].ident, Args: args, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1540: + case 1541: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-3].expr}, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1541: + case 1542: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-5].ident, Args: []ast.ExprNode{yyS[yypt-3].expr}, IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1542: + case 1543: { parser.yyVAL.expr = &ast.WindowFuncExpr{F: yyS[yypt-8].ident, Args: []ast.ExprNode{yyS[yypt-6].expr, yyS[yypt-4].expr}, FromLast: yyS[yypt-2].item.(bool), IgnoreNull: yyS[yypt-1].item.(bool), Spec: yyS[yypt-0].item.(ast.WindowSpec)} } - case 1543: + case 1544: { parser.yyVAL.item = nil } - case 1544: + case 1545: { args := []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -16877,7 +16889,7 @@ yynewstate: } parser.yyVAL.item = args } - case 1545: + case 1546: { args := []ast.ExprNode{ast.NewValueExpr(yyS[yypt-1].item, parser.charset, parser.collation)} if yyS[yypt-0].item != nil { @@ -16885,17 +16897,13 @@ yynewstate: } parser.yyVAL.item = args } - case 1546: - { - parser.yyVAL.item = nil - } case 1547: { - parser.yyVAL.item = yyS[yypt-0].expr + parser.yyVAL.item = nil } case 1548: { - parser.yyVAL.item = false + parser.yyVAL.item = yyS[yypt-0].expr } case 1549: { @@ -16903,11 +16911,11 @@ yynewstate: } case 1550: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 1551: { - parser.yyVAL.item = false + parser.yyVAL.item = true } case 1552: { @@ -16915,13 +16923,17 @@ yynewstate: } case 1553: { - parser.yyVAL.item = true + parser.yyVAL.item = false } case 1554: { - parser.yyVAL.item = &ast.TableRefsClause{TableRefs: yyS[yypt-0].item.(*ast.Join)} + parser.yyVAL.item = true } case 1555: + { + parser.yyVAL.item = &ast.TableRefsClause{TableRefs: yyS[yypt-0].item.(*ast.Join)} + } + case 1556: { if j, ok := yyS[yypt-0].item.(*ast.Join); ok { // if $1 is Join, use it directly @@ -16930,12 +16942,12 @@ yynewstate: parser.yyVAL.item = &ast.Join{Left: yyS[yypt-0].item.(ast.ResultSetNode), Right: nil} } } - case 1556: + case 1557: { /* from a, b is default cross join */ parser.yyVAL.item = &ast.Join{Left: yyS[yypt-2].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), Tp: ast.CrossJoin} } - case 1558: + case 1559: { /* * ODBC escape syntax for outer join is { OJ join_table } @@ -16943,7 +16955,7 @@ yynewstate: */ parser.yyVAL.item = yyS[yypt-1].item } - case 1561: + case 1562: { tn := yyS[yypt-5].item.(*ast.TableName) tn.PartitionNames = yyS[yypt-4].item.([]model.CIStr) @@ -16956,66 +16968,66 @@ yynewstate: } parser.yyVAL.item = &ast.TableSource{Source: tn, AsName: yyS[yypt-3].item.(model.CIStr)} } - case 1562: + case 1563: { resultNode := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query parser.yyVAL.item = &ast.TableSource{Source: resultNode, AsName: yyS[yypt-0].item.(model.CIStr)} } - case 1563: + case 1564: { j := yyS[yypt-1].item.(*ast.Join) j.ExplicitParens = true parser.yyVAL.item = yyS[yypt-1].item } - case 1564: + case 1565: { parser.yyVAL.item = []model.CIStr{} } - case 1565: + case 1566: { parser.yyVAL.item = yyS[yypt-1].item } - case 1566: + case 1567: { parser.yyVAL.item = model.CIStr{} } - case 1568: + case 1569: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1569: + case 1570: { parser.yyVAL.item = model.NewCIStr(yyS[yypt-0].ident) } - case 1570: + case 1571: { parser.yyVAL.item = ast.HintUse } - case 1571: + case 1572: { parser.yyVAL.item = ast.HintIgnore } - case 1572: + case 1573: { parser.yyVAL.item = ast.HintForce } - case 1573: + case 1574: { parser.yyVAL.item = ast.HintForScan } - case 1574: + case 1575: { parser.yyVAL.item = ast.HintForJoin } - case 1575: + case 1576: { parser.yyVAL.item = ast.HintForOrderBy } - case 1576: + case 1577: { parser.yyVAL.item = ast.HintForGroupBy } - case 1577: + case 1578: { parser.yyVAL.item = &ast.IndexHint{ IndexNames: yyS[yypt-1].item.([]model.CIStr), @@ -17023,134 +17035,134 @@ yynewstate: HintScope: yyS[yypt-3].item.(ast.IndexHintScope), } } - case 1578: + case 1579: { var nameList []model.CIStr parser.yyVAL.item = nameList } - case 1579: + case 1580: { parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 1580: + case 1581: { parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) } - case 1581: + case 1582: { parser.yyVAL.item = []model.CIStr{model.NewCIStr(yyS[yypt-0].ident)} } - case 1582: + case 1583: { parser.yyVAL.item = append(yyS[yypt-2].item.([]model.CIStr), model.NewCIStr(yyS[yypt-0].ident)) } - case 1583: + case 1584: { parser.yyVAL.item = []*ast.IndexHint{yyS[yypt-0].item.(*ast.IndexHint)} } - case 1584: + case 1585: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.IndexHint), yyS[yypt-0].item.(*ast.IndexHint)) } - case 1585: + case 1586: { parser.yyVAL.item = []*ast.IndexHint{} } - case 1587: + case 1588: { parser.yyVAL.item = ast.NewCrossJoin(yyS[yypt-2].item.(ast.ResultSetNode), yyS[yypt-0].item.(ast.ResultSetNode)) } - case 1588: + case 1589: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-4].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), Tp: ast.CrossJoin, On: on} } - case 1589: + case 1590: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-6].item.(ast.ResultSetNode), Right: yyS[yypt-4].item.(ast.ResultSetNode), Tp: ast.CrossJoin, Using: yyS[yypt-1].item.([]*ast.ColumnName)} } - case 1590: + case 1591: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-6].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), Tp: yyS[yypt-5].item.(ast.JoinType), On: on} } - case 1591: + case 1592: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-8].item.(ast.ResultSetNode), Right: yyS[yypt-4].item.(ast.ResultSetNode), Tp: yyS[yypt-7].item.(ast.JoinType), Using: yyS[yypt-1].item.([]*ast.ColumnName)} } - case 1592: + case 1593: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-3].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), NaturalJoin: true} } - case 1593: + case 1594: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-5].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), Tp: yyS[yypt-3].item.(ast.JoinType), NaturalJoin: true} } - case 1594: + case 1595: { parser.yyVAL.item = &ast.Join{Left: yyS[yypt-2].item.(ast.ResultSetNode), Right: yyS[yypt-0].item.(ast.ResultSetNode), StraightJoin: true} } - case 1595: + case 1596: { on := &ast.OnCondition{Expr: yyS[yypt-0].expr} parser.yyVAL.item = &ast.Join{Left: yyS[yypt-4].item.(ast.ResultSetNode), Right: yyS[yypt-2].item.(ast.ResultSetNode), StraightJoin: true, On: on} } - case 1596: + case 1597: { parser.yyVAL.item = ast.LeftJoin } - case 1597: + case 1598: { parser.yyVAL.item = ast.RightJoin } - case 1603: + case 1604: { parser.yyVAL.item = nil } - case 1604: + case 1605: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-0].item.(ast.ValueExpr)} } - case 1605: + case 1606: { parser.yyVAL.item = ast.NewValueExpr(yyS[yypt-0].item, parser.charset, parser.collation) } - case 1606: + case 1607: { parser.yyVAL.item = ast.NewParamMarkerExpr(yyS[yypt].offset) } - case 1611: + case 1612: { parser.yyVAL.item = ast.NewValueExpr(uint64(1), parser.charset, parser.collation) } - case 1613: + case 1614: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-0].item.(ast.ExprNode)} } - case 1614: + case 1615: { parser.yyVAL.item = &ast.Limit{Offset: yyS[yypt-2].item.(ast.ExprNode), Count: yyS[yypt-0].item.(ast.ExprNode)} } - case 1615: + case 1616: { parser.yyVAL.item = &ast.Limit{Offset: yyS[yypt-0].item.(ast.ExprNode), Count: yyS[yypt-2].item.(ast.ExprNode)} } - case 1616: + case 1617: { parser.yyVAL.item = &ast.Limit{Count: yyS[yypt-2].item.(ast.ExprNode)} } - case 1617: + case 1618: { parser.yyVAL.item = nil } - case 1619: + case 1620: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.TableHints = yyS[yypt-0].item.([]*ast.TableOptimizerHint) parser.yyVAL.item = opt } - case 1620: + case 1621: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true @@ -17162,61 +17174,61 @@ yynewstate: } parser.yyVAL.item = opt } - case 1621: + case 1622: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.Priority = yyS[yypt-0].item.(mysql.PriorityEnum) parser.yyVAL.item = opt } - case 1622: + case 1623: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLSmallResult = true parser.yyVAL.item = opt } - case 1623: + case 1624: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLBigResult = true parser.yyVAL.item = opt } - case 1624: + case 1625: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.SQLBufferResult = true parser.yyVAL.item = opt } - case 1625: + case 1626: { opt := &ast.SelectStmtOpts{} opt.SQLCache = yyS[yypt-0].item.(bool) parser.yyVAL.item = opt } - case 1626: + case 1627: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.CalcFoundRows = true parser.yyVAL.item = opt } - case 1627: + case 1628: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true opt.StraightJoin = true parser.yyVAL.item = opt } - case 1628: + case 1629: { opt := &ast.SelectStmtOpts{} opt.SQLCache = true parser.yyVAL.item = opt } - case 1630: + case 1631: { opts := yyS[yypt-1].item.(*ast.SelectStmtOpts) opt := yyS[yypt-0].item.(*ast.SelectStmtOpts) @@ -17261,7 +17273,7 @@ yynewstate: parser.yyVAL.item = opts } - case 1632: + case 1633: { hints, warns := parser.parseHint(yyS[yypt-0].ident) for _, w := range warns { @@ -17270,31 +17282,31 @@ yynewstate: } parser.yyVAL.item = hints } - case 1633: + case 1634: { parser.yyVAL.item = nil } - case 1635: + case 1636: { parser.yyVAL.item = true } - case 1636: + case 1637: { parser.yyVAL.item = false } - case 1637: + case 1638: { parser.yyVAL.item = &ast.FieldList{Fields: yyS[yypt-0].item.([]*ast.SelectField)} } - case 1638: + case 1639: { parser.yyVAL.item = nil } - case 1640: + case 1641: { parser.yyVAL.item = nil } - case 1641: + case 1642: { x := &ast.SelectIntoOption{ Tp: ast.SelectIntoOutfile, @@ -17309,7 +17321,7 @@ yynewstate: parser.yyVAL.item = x } - case 1642: + case 1643: { rs := yyS[yypt-1].statement.(*ast.SelectStmt) endOffset := parser.endOffset(&yyS[yypt]) @@ -17319,14 +17331,14 @@ yynewstate: rs.SetText(parser.lexer.client, src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1643: + case 1644: { rs := yyS[yypt-1].statement.(*ast.SetOprStmt) src := parser.src rs.SetText(parser.lexer.client, src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1644: + case 1645: { rs := yyS[yypt-1].statement.(*ast.SelectStmt) endOffset := parser.endOffset(&yyS[yypt]) @@ -17336,7 +17348,7 @@ yynewstate: rs.SetText(parser.lexer.client, src[yyS[yypt-1].offset:yyS[yypt].offset]) parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } - case 1645: + case 1646: { subQuery := yyS[yypt-1].expr.(*ast.SubqueryExpr).Query isRecursive := true @@ -17359,32 +17371,32 @@ yynewstate: parser.yyVAL.expr = &ast.SubqueryExpr{Query: rs} } } - case 1646: + case 1647: { parser.yyVAL.item = nil } - case 1647: + case 1648: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdate, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1648: + case 1649: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShare, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1649: + case 1650: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateNoWait, Tables: yyS[yypt-1].item.([]*ast.TableName), } } - case 1650: + case 1651: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateWaitN, @@ -17392,55 +17404,55 @@ yynewstate: Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1651: + case 1652: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShareNoWait, Tables: yyS[yypt-1].item.([]*ast.TableName), } } - case 1652: + case 1653: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdateSkipLocked, Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1653: + case 1654: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShareSkipLocked, Tables: yyS[yypt-2].item.([]*ast.TableName), } } - case 1654: + case 1655: { parser.yyVAL.item = &ast.SelectLockInfo{ LockType: ast.SelectLockForShare, Tables: []*ast.TableName{}, } } - case 1655: + case 1656: { parser.yyVAL.item = []*ast.TableName{} } - case 1656: + case 1657: { parser.yyVAL.item = yyS[yypt-0].item.([]*ast.TableName) } - case 1659: + case 1660: { setOpr := yyS[yypt-0].statement.(*ast.SetOprStmt) setOpr.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = setOpr } - case 1660: + case 1661: { setOpr := yyS[yypt-0].statement.(*ast.SetOprStmt) setOpr.With = yyS[yypt-1].item.(*ast.WithClause) parser.yyVAL.statement = setOpr } - case 1661: + case 1662: { setOprList1 := yyS[yypt-2].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17457,7 +17469,7 @@ yynewstate: setOpr.SelectList.Selects = append(setOpr.SelectList.Selects, st) parser.yyVAL.statement = setOpr } - case 1662: + case 1663: { setOprList1 := yyS[yypt-2].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17480,7 +17492,7 @@ yynewstate: setOpr := &ast.SetOprStmt{SelectList: &ast.SetOprSelectList{Selects: setOprList}} parser.yyVAL.statement = setOpr } - case 1663: + case 1664: { setOprList1 := yyS[yypt-3].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17504,7 +17516,7 @@ yynewstate: setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause) parser.yyVAL.statement = setOpr } - case 1664: + case 1665: { setOprList1 := yyS[yypt-3].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17528,7 +17540,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1665: + case 1666: { setOprList1 := yyS[yypt-4].item.([]ast.Node) if sel, isSelect := setOprList1[len(setOprList1)-1].(*ast.SelectStmt); isSelect && !sel.IsInBraces { @@ -17553,7 +17565,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1666: + case 1667: { var setOprList []ast.Node var with *ast.WithClause @@ -17569,7 +17581,7 @@ yynewstate: setOpr.OrderBy = yyS[yypt-0].item.(*ast.OrderByClause) parser.yyVAL.statement = setOpr } - case 1667: + case 1668: { var setOprList []ast.Node var with *ast.WithClause @@ -17585,7 +17597,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1668: + case 1669: { var setOprList []ast.Node var with *ast.WithClause @@ -17602,7 +17614,7 @@ yynewstate: setOpr.Limit = yyS[yypt-0].item.(*ast.Limit) parser.yyVAL.statement = setOpr } - case 1670: + case 1671: { setOprList1 := yyS[yypt-2].item.([]ast.Node) setOprList2 := yyS[yypt-0].item.([]ast.Node) @@ -17618,11 +17630,11 @@ yynewstate: } parser.yyVAL.item = append(setOprList1, setOprList2...) } - case 1671: + case 1672: { parser.yyVAL.item = []ast.Node{yyS[yypt-0].statement.(*ast.SelectStmt)} } - case 1672: + case 1673: { var setOprList []ast.Node switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -17633,7 +17645,7 @@ yynewstate: } parser.yyVAL.item = setOprList } - case 1673: + case 1674: { var tp ast.SetOprType tp = ast.Union @@ -17642,7 +17654,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1674: + case 1675: { var tp ast.SetOprType tp = ast.Except @@ -17651,7 +17663,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1675: + case 1676: { var tp ast.SetOprType tp = ast.Intersect @@ -17660,7 +17672,7 @@ yynewstate: } parser.yyVAL.item = &tp } - case 1677: + case 1678: { parser.yyVAL.statement = &ast.ChangeStmt{ NodeType: ast.PumpType, @@ -17668,7 +17680,7 @@ yynewstate: NodeID: yyS[yypt-0].ident, } } - case 1678: + case 1679: { parser.yyVAL.statement = &ast.ChangeStmt{ NodeType: ast.DrainerType, @@ -17676,19 +17688,19 @@ yynewstate: NodeID: yyS[yypt-0].ident, } } - case 1679: + case 1680: { parser.yyVAL.statement = &ast.SetStmt{Variables: yyS[yypt-0].item.([]*ast.VariableAssignment)} } - case 1680: + case 1681: { parser.yyVAL.statement = &ast.SetPwdStmt{Password: yyS[yypt-0].ident} } - case 1681: + case 1682: { parser.yyVAL.statement = &ast.SetPwdStmt{User: yyS[yypt-2].item.(*auth.UserIdentity), Password: yyS[yypt-0].ident} } - case 1682: + case 1683: { vars := yyS[yypt-0].item.([]*ast.VariableAssignment) for _, v := range vars { @@ -17696,11 +17708,11 @@ yynewstate: } parser.yyVAL.statement = &ast.SetStmt{Variables: vars} } - case 1683: + case 1684: { parser.yyVAL.statement = &ast.SetStmt{Variables: yyS[yypt-0].item.([]*ast.VariableAssignment)} } - case 1684: + case 1685: { assigns := yyS[yypt-0].item.([]*ast.VariableAssignment) for i := 0; i < len(assigns); i++ { @@ -17711,19 +17723,19 @@ yynewstate: } parser.yyVAL.statement = &ast.SetStmt{Variables: assigns} } - case 1685: + case 1686: { parser.yyVAL.statement = &ast.SetConfigStmt{Type: strings.ToLower(yyS[yypt-3].ident), Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr} } - case 1686: + case 1687: { parser.yyVAL.statement = &ast.SetConfigStmt{Instance: yyS[yypt-3].ident, Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr} } - case 1687: + case 1688: { parser.yyVAL.statement = yyS[yypt-0].item.(*ast.SetRoleStmt) } - case 1688: + case 1689: { tmp := yyS[yypt-2].item.(*ast.SetRoleStmt) parser.yyVAL.statement = &ast.SetDefaultRoleStmt{ @@ -17732,27 +17744,27 @@ yynewstate: UserList: yyS[yypt-0].item.([]*auth.UserIdentity), } } - case 1689: + case 1690: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleNone, RoleList: nil} } - case 1690: + case 1691: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleAll, RoleList: nil} } - case 1691: + case 1692: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleRegular, RoleList: yyS[yypt-0].item.([]*auth.RoleIdentity)} } - case 1692: + case 1693: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleAllExcept, RoleList: yyS[yypt-0].item.([]*auth.RoleIdentity)} } - case 1694: + case 1695: { parser.yyVAL.item = &ast.SetRoleStmt{SetRoleOpt: ast.SetRoleDefault, RoleList: nil} } - case 1695: + case 1696: { if yyS[yypt-0].item != nil { parser.yyVAL.item = yyS[yypt-0].item @@ -17760,7 +17772,7 @@ yynewstate: parser.yyVAL.item = []*ast.VariableAssignment{} } } - case 1696: + case 1697: { if yyS[yypt-0].item != nil { varAssigns := yyS[yypt-0].item.([]*ast.VariableAssignment) @@ -17769,28 +17781,28 @@ yynewstate: parser.yyVAL.item = yyS[yypt-2].item } } - case 1697: + case 1698: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr(yyS[yypt-0].ident, parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_isolation", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1698: + case 1699: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr("0", parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_read_only", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1699: + case 1700: { varAssigns := []*ast.VariableAssignment{} expr := ast.NewValueExpr("1", parser.charset, parser.collation) varAssigns = append(varAssigns, &ast.VariableAssignment{Name: "tx_read_only", Value: expr, IsSystem: true}) parser.yyVAL.item = varAssigns } - case 1700: + case 1701: { varAssigns := []*ast.VariableAssignment{} asof := yyS[yypt-0].item.(*ast.AsOfClause) @@ -17799,59 +17811,59 @@ yynewstate: } parser.yyVAL.item = varAssigns } - case 1701: + case 1702: { parser.yyVAL.ident = ast.RepeatableRead } - case 1702: + case 1703: { parser.yyVAL.ident = ast.ReadCommitted } - case 1703: + case 1704: { parser.yyVAL.ident = ast.ReadUncommitted } - case 1704: + case 1705: { parser.yyVAL.ident = ast.Serializable } - case 1705: + case 1706: { parser.yyVAL.expr = ast.NewValueExpr("ON", parser.charset, parser.collation) } - case 1706: + case 1707: { parser.yyVAL.expr = ast.NewValueExpr("BINARY", parser.charset, parser.collation) } - case 1711: + case 1712: { parser.yyVAL.ident = yyS[yypt-2].ident + "." + yyS[yypt-0].ident } - case 1713: + case 1714: { parser.yyVAL.ident = yyS[yypt-2].ident + "." + yyS[yypt-0].ident } - case 1714: + case 1715: { parser.yyVAL.ident = yyS[yypt-2].ident + "-" + yyS[yypt-0].ident } - case 1715: + case 1716: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1716: + case 1717: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsGlobal: true, IsSystem: true} } - case 1717: + case 1718: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1718: + case 1719: { parser.yyVAL.item = &ast.VariableAssignment{Name: yyS[yypt-2].ident, Value: yyS[yypt-0].expr, IsSystem: true} } - case 1719: + case 1720: { v := strings.ToLower(yyS[yypt-2].ident) var isGlobal bool @@ -17867,27 +17879,27 @@ yynewstate: } parser.yyVAL.item = &ast.VariableAssignment{Name: v, Value: yyS[yypt-0].expr, IsGlobal: isGlobal, IsSystem: true} } - case 1720: + case 1721: { v := yyS[yypt-2].ident v = strings.TrimPrefix(v, "@") parser.yyVAL.item = &ast.VariableAssignment{Name: v, Value: yyS[yypt-0].expr} } - case 1721: + case 1722: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, Value: ast.NewValueExpr(yyS[yypt-0].ident, "", ""), } } - case 1722: + case 1723: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, Value: ast.NewValueExpr(yyS[yypt-2].ident, "", ""), } } - case 1723: + case 1724: { parser.yyVAL.item = &ast.VariableAssignment{ Name: ast.SetNames, @@ -17895,24 +17907,24 @@ yynewstate: ExtendValue: ast.NewValueExpr(yyS[yypt-0].ident, "", ""), } } - case 1724: + case 1725: { v := &ast.DefaultExpr{} parser.yyVAL.item = &ast.VariableAssignment{Name: ast.SetNames, Value: v} } - case 1725: + case 1726: { parser.yyVAL.item = &ast.VariableAssignment{Name: ast.SetCharset, Value: yyS[yypt-0].expr} } - case 1726: + case 1727: { parser.yyVAL.expr = ast.NewValueExpr(yyS[yypt-0].ident, "", "") } - case 1727: + case 1728: { parser.yyVAL.expr = &ast.DefaultExpr{} } - case 1728: + case 1729: { // Validate input charset name to keep the same behavior as parser of MySQL. cs, err := charset.GetCharsetInfo(yyS[yypt-0].ident) @@ -17924,11 +17936,11 @@ yynewstate: // to keep lower case of input for generated column restore. parser.yyVAL.ident = cs.Name } - case 1729: + case 1730: { parser.yyVAL.ident = charset.CharsetBin } - case 1730: + case 1731: { info, err := charset.GetCollationByName(yyS[yypt-0].ident) if err != nil { @@ -17937,19 +17949,19 @@ yynewstate: } parser.yyVAL.ident = info.Name } - case 1731: + case 1732: { parser.yyVAL.ident = charset.CollationBin } - case 1732: + case 1733: { parser.yyVAL.item = []*ast.VariableAssignment{yyS[yypt-0].item.(*ast.VariableAssignment)} } - case 1733: + case 1734: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.VariableAssignment), yyS[yypt-0].item.(*ast.VariableAssignment)) } - case 1736: + case 1737: { v := strings.ToLower(yyS[yypt-0].ident) var isGlobal bool @@ -17966,77 +17978,77 @@ yynewstate: } parser.yyVAL.expr = &ast.VariableExpr{Name: v, IsGlobal: isGlobal, IsSystem: true, ExplicitScope: explicitScope} } - case 1737: + case 1738: { v := yyS[yypt-0].ident v = strings.TrimPrefix(v, "@") parser.yyVAL.expr = &ast.VariableExpr{Name: v, IsGlobal: false, IsSystem: false} } - case 1738: + case 1739: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1739: + case 1740: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-2].ident, Hostname: yyS[yypt-0].ident} } - case 1740: + case 1741: { parser.yyVAL.item = &auth.UserIdentity{Username: yyS[yypt-1].ident, Hostname: strings.TrimPrefix(yyS[yypt-0].ident, "@")} } - case 1741: + case 1742: { parser.yyVAL.item = &auth.UserIdentity{CurrentUser: true} } - case 1742: + case 1743: { parser.yyVAL.item = []*auth.UserIdentity{yyS[yypt-0].item.(*auth.UserIdentity)} } - case 1743: + case 1744: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*auth.UserIdentity), yyS[yypt-0].item.(*auth.UserIdentity)) } - case 1745: + case 1746: { parser.yyVAL.ident = yyS[yypt-1].ident } - case 1749: + case 1750: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-2].ident, Hostname: yyS[yypt-0].ident} } - case 1750: + case 1751: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-1].ident, Hostname: strings.TrimPrefix(yyS[yypt-0].ident, "@")} } - case 1751: + case 1752: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1752: + case 1753: { parser.yyVAL.item = yyS[yypt-0].item } - case 1753: + case 1754: { parser.yyVAL.item = &auth.RoleIdentity{Username: yyS[yypt-0].ident, Hostname: "%"} } - case 1754: + case 1755: { parser.yyVAL.item = yyS[yypt-0].item } - case 1755: + case 1756: { parser.yyVAL.item = []*auth.RoleIdentity{yyS[yypt-0].item.(*auth.RoleIdentity)} } - case 1756: + case 1757: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*auth.RoleIdentity), yyS[yypt-0].item.(*auth.RoleIdentity)) } - case 1757: + case 1758: { parser.yyVAL.statement = &ast.AdminStmt{Tp: ast.AdminShowDDL} } - case 1758: + case 1759: { stmt := &ast.AdminStmt{Tp: ast.AdminShowDDLJobs} if yyS[yypt-0].item != nil { @@ -18044,7 +18056,7 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1759: + case 1760: { stmt := &ast.AdminStmt{ Tp: ast.AdminShowDDLJobs, @@ -18055,21 +18067,21 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1760: + case 1761: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowNextRowID, Tables: []*ast.TableName{yyS[yypt-1].item.(*ast.TableName)}, } } - case 1761: + case 1762: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckTable, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1762: + case 1763: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckIndex, @@ -18077,7 +18089,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1763: + case 1764: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminRecoverIndex, @@ -18085,7 +18097,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1764: + case 1765: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCleanupIndex, @@ -18093,7 +18105,7 @@ yynewstate: Index: string(yyS[yypt-0].ident), } } - case 1765: + case 1766: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCheckIndexRange, @@ -18102,136 +18114,136 @@ yynewstate: HandleRanges: yyS[yypt-0].item.([]ast.HandleRange), } } - case 1766: + case 1767: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminChecksumTable, Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1767: + case 1768: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCancelDDLJobs, JobIDs: yyS[yypt-0].item.([]int64), } } - case 1768: + case 1769: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowDDLJobQueries, JobIDs: yyS[yypt-0].item.([]int64), } } - case 1769: + case 1770: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowSlow, ShowSlow: yyS[yypt-0].item.(*ast.ShowSlow), } } - case 1770: + case 1771: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadExprPushdownBlacklist, } } - case 1771: + case 1772: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadOptRuleBlacklist, } } - case 1772: + case 1773: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminPluginEnable, Plugins: yyS[yypt-0].item.([]string), } } - case 1773: + case 1774: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminPluginDisable, Plugins: yyS[yypt-0].item.([]string), } } - case 1774: + case 1775: { parser.yyVAL.statement = &ast.CleanupTableLockStmt{ Tables: yyS[yypt-0].item.([]*ast.TableName), } } - case 1775: + case 1776: { parser.yyVAL.statement = &ast.RepairTableStmt{ Table: yyS[yypt-1].item.(*ast.TableName), CreateStmt: yyS[yypt-0].statement.(*ast.CreateTableStmt), } } - case 1776: + case 1777: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminFlushBindings, } } - case 1777: + case 1778: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminCaptureBindings, } } - case 1778: + case 1779: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminEvolveBindings, } } - case 1779: + case 1780: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadBindings, } } - case 1780: + case 1781: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadStatistics, } } - case 1781: + case 1782: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminReloadStatistics, } } - case 1782: + case 1783: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminShowTelemetry, } } - case 1783: + case 1784: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminResetTelemetryID, } } - case 1784: + case 1785: { parser.yyVAL.statement = &ast.AdminStmt{ Tp: ast.AdminFlushPlanCache, StatementScope: yyS[yypt-1].item.(ast.StatementScope), } } - case 1785: + case 1786: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowRecent, Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1786: + case 1787: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18239,7 +18251,7 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1787: + case 1788: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18247,7 +18259,7 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1788: + case 1789: { parser.yyVAL.item = &ast.ShowSlow{ Tp: ast.ShowSlowTop, @@ -18255,27 +18267,27 @@ yynewstate: Count: getUint64FromNUM(yyS[yypt-0].item), } } - case 1789: + case 1790: { parser.yyVAL.item = []ast.HandleRange{yyS[yypt-0].item.(ast.HandleRange)} } - case 1790: + case 1791: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.HandleRange), yyS[yypt-0].item.(ast.HandleRange)) } - case 1791: + case 1792: { parser.yyVAL.item = ast.HandleRange{Begin: yyS[yypt-3].item.(int64), End: yyS[yypt-1].item.(int64)} } - case 1792: + case 1793: { parser.yyVAL.item = []int64{yyS[yypt-0].item.(int64)} } - case 1793: + case 1794: { parser.yyVAL.item = append(yyS[yypt-2].item.([]int64), yyS[yypt-0].item.(int64)) } - case 1794: + case 1795: { stmt := yyS[yypt-1].item.(*ast.ShowStmt) if yyS[yypt-0].item != nil { @@ -18287,21 +18299,21 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1795: + case 1796: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateTable, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1796: + case 1797: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateView, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1797: + case 1798: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateDatabase, @@ -18309,21 +18321,21 @@ yynewstate: DBName: yyS[yypt-0].ident, } } - case 1798: + case 1799: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateSequence, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1799: + case 1800: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreatePlacementPolicy, DBName: yyS[yypt-0].ident, } } - case 1800: + case 1801: { // See https://dev.mysql.com/doc/refman/5.7/en/show-create-user.html parser.yyVAL.statement = &ast.ShowStmt{ @@ -18331,14 +18343,14 @@ yynewstate: User: yyS[yypt-0].item.(*auth.UserIdentity), } } - case 1801: + case 1802: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowCreateImport, DBName: yyS[yypt-0].ident, // we reuse DBName of ShowStmt } } - case 1802: + case 1803: { stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, @@ -18350,14 +18362,14 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1803: + case 1804: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowTableNextRowId, Table: yyS[yypt-1].item.(*ast.TableName), } } - case 1804: + case 1805: { stmt := &ast.ShowStmt{ Tp: ast.ShowRegions, @@ -18370,12 +18382,12 @@ yynewstate: } parser.yyVAL.statement = stmt } - case 1805: + case 1806: { // See https://dev.mysql.com/doc/refman/5.7/en/show-grants.html parser.yyVAL.statement = &ast.ShowStmt{Tp: ast.ShowGrants} } - case 1806: + case 1807: { // See https://dev.mysql.com/doc/refman/5.7/en/show-grants.html if yyS[yypt-0].item != nil { @@ -18392,26 +18404,26 @@ yynewstate: } } } - case 1807: + case 1808: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowMasterStatus, } } - case 1808: + case 1809: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowProcessList, Full: yyS[yypt-1].item.(bool), } } - case 1809: + case 1810: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowProfiles, } } - case 1810: + case 1811: { v := &ast.ShowStmt{ Tp: ast.ShowProfile, @@ -18427,37 +18439,37 @@ yynewstate: } parser.yyVAL.statement = v } - case 1811: + case 1812: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowPrivileges, } } - case 1812: + case 1813: { parser.yyVAL.statement = &ast.ShowStmt{ Tp: ast.ShowBuiltins, } } - case 1813: + case 1814: { parser.yyVAL.statement = yyS[yypt-0].item.(*ast.ShowStmt) } - case 1814: + case 1815: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForDatabase, DBName: yyS[yypt-0].ident, } } - case 1815: + case 1816: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForTable, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1816: + case 1817: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlacementForPartition, @@ -18465,90 +18477,90 @@ yynewstate: Partition: model.NewCIStr(yyS[yypt-0].ident), } } - case 1817: + case 1818: { parser.yyVAL.item = nil } - case 1819: + case 1820: { parser.yyVAL.item = []int{yyS[yypt-0].item.(int)} } - case 1820: + case 1821: { l := yyS[yypt-2].item.([]int) l = append(l, yyS[yypt-0].item.(int)) parser.yyVAL.item = l } - case 1821: + case 1822: { parser.yyVAL.item = ast.ProfileTypeCPU } - case 1822: + case 1823: { parser.yyVAL.item = ast.ProfileTypeMemory } - case 1823: + case 1824: { parser.yyVAL.item = ast.ProfileTypeBlockIo } - case 1824: + case 1825: { parser.yyVAL.item = ast.ProfileTypeContextSwitch } - case 1825: + case 1826: { parser.yyVAL.item = ast.ProfileTypePageFaults } - case 1826: + case 1827: { parser.yyVAL.item = ast.ProfileTypeIpc } - case 1827: + case 1828: { parser.yyVAL.item = ast.ProfileTypeSwaps } - case 1828: + case 1829: { parser.yyVAL.item = ast.ProfileTypeSource } - case 1829: + case 1830: { parser.yyVAL.item = ast.ProfileTypeAll } - case 1830: + case 1831: { parser.yyVAL.item = nil } - case 1831: + case 1832: { v := yyS[yypt-0].item.(int64) parser.yyVAL.item = &v } - case 1832: + case 1833: { parser.yyVAL.item = nil } - case 1833: + case 1834: { parser.yyVAL.item = yyS[yypt-0].item.([]*auth.RoleIdentity) } - case 1839: + case 1840: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowEngines} } - case 1840: + case 1841: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowDatabases} } - case 1841: + case 1842: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowConfig} } - case 1842: + case 1843: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowCharset} } - case 1843: + case 1844: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTables, @@ -18556,28 +18568,28 @@ yynewstate: Full: yyS[yypt-2].item.(bool), } } - case 1844: + case 1845: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowOpenTables, DBName: yyS[yypt-0].ident, } } - case 1845: + case 1846: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTableStatus, DBName: yyS[yypt-0].ident, } } - case 1846: + case 1847: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowIndex, Table: yyS[yypt-0].item.(*ast.TableName), } } - case 1847: + case 1848: { show := &ast.ShowStmt{ Tp: ast.ShowIndex, @@ -18585,7 +18597,7 @@ yynewstate: } parser.yyVAL.item = show } - case 1848: + case 1849: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowColumns, @@ -18594,7 +18606,7 @@ yynewstate: Full: yyS[yypt-3].item.(bool), } } - case 1849: + case 1850: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowColumns, @@ -18604,67 +18616,67 @@ yynewstate: Extended: true, } } - case 1850: + case 1851: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowWarnings} } - case 1851: + case 1852: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowErrors} } - case 1852: + case 1853: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowVariables, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1853: + case 1854: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowStatus, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1854: + case 1855: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowBindings, GlobalScope: yyS[yypt-1].item.(bool), } } - case 1855: + case 1856: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowCollation, } } - case 1856: + case 1857: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowTriggers, DBName: yyS[yypt-0].ident, } } - case 1857: + case 1858: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowProcedureStatus, } } - case 1858: + case 1859: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPumpStatus, } } - case 1859: + case 1860: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowDrainerStatus, } } - case 1860: + case 1861: { // This statement is similar to SHOW PROCEDURE STATUS but for stored functions. // See http://dev.mysql.com/doc/refman/5.7/en/show-function-status.html @@ -18674,185 +18686,185 @@ yynewstate: Tp: ast.ShowProcedureStatus, } } - case 1861: + case 1862: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowEvents, DBName: yyS[yypt-0].ident, } } - case 1862: + case 1863: { parser.yyVAL.item = &ast.ShowStmt{ Tp: ast.ShowPlugins, } } - case 1863: + case 1864: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsExtended} } - case 1864: + case 1865: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsMeta} } - case 1865: + case 1866: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHistograms} } - case 1866: + case 1867: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsTopN} } - case 1867: + case 1868: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsBuckets} } - case 1868: + case 1869: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowStatsHealthy} } - case 1869: + case 1870: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowHistogramsInFlight} } - case 1870: + case 1871: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowColumnStatsUsage} } - case 1871: + case 1872: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowAnalyzeStatus} } - case 1872: + case 1873: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowBackups} } - case 1873: + case 1874: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowRestores} } - case 1874: + case 1875: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowImports} } - case 1875: + case 1876: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowPlacement} } - case 1876: + case 1877: { parser.yyVAL.item = &ast.ShowStmt{Tp: ast.ShowPlacementLabels} } - case 1877: + case 1878: { parser.yyVAL.item = nil } - case 1878: + case 1879: { parser.yyVAL.item = &ast.PatternLikeExpr{ Pattern: yyS[yypt-0].expr, Escape: '\\', } } - case 1879: + case 1880: { parser.yyVAL.item = yyS[yypt-0].expr } - case 1880: + case 1881: { parser.yyVAL.item = false } - case 1881: + case 1882: { parser.yyVAL.item = true } - case 1882: + case 1883: { parser.yyVAL.item = false } - case 1883: + case 1884: { parser.yyVAL.item = ast.StatementScopeSession } - case 1884: + case 1885: { parser.yyVAL.item = ast.StatementScopeGlobal } - case 1885: + case 1886: { parser.yyVAL.item = ast.StatementScopeInstance } - case 1886: + case 1887: { parser.yyVAL.item = ast.StatementScopeSession } - case 1887: + case 1888: { parser.yyVAL.item = false } - case 1888: + case 1889: { parser.yyVAL.item = true } - case 1889: + case 1890: { parser.yyVAL.ident = "" } - case 1890: + case 1891: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 1891: + case 1892: { parser.yyVAL.item = yyS[yypt-0].item.(*ast.TableName) } - case 1892: + case 1893: { tmp := yyS[yypt-0].item.(*ast.FlushStmt) tmp.NoWriteToBinLog = yyS[yypt-1].item.(bool) parser.yyVAL.statement = tmp } - case 1893: + case 1894: { parser.yyVAL.item = []string{yyS[yypt-0].ident} } - case 1894: + case 1895: { parser.yyVAL.item = append(yyS[yypt-2].item.([]string), yyS[yypt-0].ident) } - case 1895: + case 1896: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushPrivileges, } } - case 1896: + case 1897: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushStatus, } } - case 1897: + case 1898: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushTiDBPlugin, Plugins: yyS[yypt-0].item.([]string), } } - case 1898: + case 1899: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushHosts, } } - case 1899: + case 1900: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushLogs, LogType: yyS[yypt-1].item.(ast.LogType), } } - case 1900: + case 1901: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushTables, @@ -18860,69 +18872,69 @@ yynewstate: ReadLock: yyS[yypt-0].item.(bool), } } - case 1901: + case 1902: { parser.yyVAL.item = &ast.FlushStmt{ Tp: ast.FlushClientErrorsSummary, } } - case 1902: + case 1903: { parser.yyVAL.item = ast.LogTypeDefault } - case 1903: + case 1904: { parser.yyVAL.item = ast.LogTypeBinary } - case 1904: + case 1905: { parser.yyVAL.item = ast.LogTypeEngine } - case 1905: + case 1906: { parser.yyVAL.item = ast.LogTypeError } - case 1906: + case 1907: { parser.yyVAL.item = ast.LogTypeGeneral } - case 1907: + case 1908: { parser.yyVAL.item = ast.LogTypeSlow } - case 1908: + case 1909: { parser.yyVAL.item = false } - case 1909: + case 1910: { parser.yyVAL.item = true } - case 1910: + case 1911: { parser.yyVAL.item = true } - case 1911: + case 1912: { parser.yyVAL.item = []*ast.TableName{} } - case 1913: + case 1914: { parser.yyVAL.item = []*ast.TableName{} } - case 1914: + case 1915: { parser.yyVAL.item = yyS[yypt-0].item } - case 1915: + case 1916: { parser.yyVAL.item = false } - case 1916: + case 1917: { parser.yyVAL.item = true } - case 1985: + case 1986: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18935,7 +18947,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 2009: + case 2010: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18948,7 +18960,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 2022: + case 2023: { var sel ast.StmtNode switch x := yyS[yypt-0].expr.(*ast.SubqueryExpr).Query.(type) { @@ -18961,7 +18973,7 @@ yynewstate: } parser.yyVAL.statement = sel } - case 2024: + case 2025: { if yyS[yypt-0].statement != nil { s := yyS[yypt-0].statement @@ -18971,7 +18983,7 @@ yynewstate: parser.result = append(parser.result, s) } } - case 2025: + case 2026: { if yyS[yypt-0].statement != nil { s := yyS[yypt-0].statement @@ -18981,7 +18993,7 @@ yynewstate: parser.result = append(parser.result, s) } } - case 2026: + case 2027: { cst := yyS[yypt-0].item.(*ast.Constraint) if yyS[yypt-1].item != nil { @@ -18989,7 +19001,7 @@ yynewstate: } parser.yyVAL.item = cst } - case 2031: + case 2032: { if yyS[yypt-0].item != nil { parser.yyVAL.item = []interface{}{yyS[yypt-0].item.(interface{})} @@ -18997,7 +19009,7 @@ yynewstate: parser.yyVAL.item = []interface{}{} } } - case 2032: + case 2033: { if yyS[yypt-0].item != nil { parser.yyVAL.item = append(yyS[yypt-2].item.([]interface{}), yyS[yypt-0].item) @@ -19005,7 +19017,7 @@ yynewstate: parser.yyVAL.item = yyS[yypt-2].item } } - case 2033: + case 2034: { var columnDefs []*ast.ColumnDef var constraints []*ast.Constraint @@ -19014,7 +19026,7 @@ yynewstate: Constraints: constraints, } } - case 2034: + case 2035: { tes := yyS[yypt-1].item.([]interface{}) var columnDefs []*ast.ColumnDef @@ -19032,69 +19044,69 @@ yynewstate: Constraints: constraints, } } - case 2036: + case 2037: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCharset, StrValue: yyS[yypt-0].ident, UintValue: ast.TableOptionCharsetWithoutConvertTo} } - case 2037: + case 2038: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCollate, StrValue: yyS[yypt-0].ident, UintValue: ast.TableOptionCharsetWithoutConvertTo} } - case 2038: + case 2039: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoIncrement, UintValue: yyS[yypt-0].item.(uint64), BoolValue: yyS[yypt-3].item.(bool)} } - case 2039: + case 2040: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoIdCache, UintValue: yyS[yypt-0].item.(uint64)} } - case 2040: + case 2041: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAutoRandomBase, UintValue: yyS[yypt-0].item.(uint64), BoolValue: yyS[yypt-3].item.(bool)} } - case 2041: + case 2042: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionAvgRowLength, UintValue: yyS[yypt-0].item.(uint64)} } - case 2042: + case 2043: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionConnection, StrValue: yyS[yypt-0].ident} } - case 2043: + case 2044: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCheckSum, UintValue: yyS[yypt-0].item.(uint64)} } - case 2044: + case 2045: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionTableCheckSum, UintValue: yyS[yypt-0].item.(uint64)} } - case 2045: + case 2046: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionPassword, StrValue: yyS[yypt-0].ident} } - case 2046: + case 2047: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionCompression, StrValue: yyS[yypt-0].ident} } - case 2047: + case 2048: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionKeyBlockSize, UintValue: yyS[yypt-0].item.(uint64)} } - case 2048: + case 2049: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionDelayKeyWrite, UintValue: yyS[yypt-0].item.(uint64)} } - case 2049: + case 2050: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionRowFormat, UintValue: yyS[yypt-0].item.(uint64)} } - case 2050: + case 2051: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionStatsPersistent} } - case 2051: + case 2052: { n := yyS[yypt-0].item.(uint64) if n != 0 && n != 1 { @@ -19105,13 +19117,13 @@ yynewstate: yylex.AppendError(yylex.Errorf("The STATS_AUTO_RECALC is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 2052: + case 2053: { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionStatsAutoRecalc, Default: true} yylex.AppendError(yylex.Errorf("The STATS_AUTO_RECALC is parsed but ignored by all storage engines.")) parser.lastErrorAsWarn() } - case 2053: + case 2054: { // Parse it but will ignore it. // In MySQL, STATS_SAMPLE_PAGES=N(Where 0 1 { @@ -20784,7 +20796,7 @@ yynewstate: OptEnclosed: true, } } - case 2372: + case 2373: { str := yyS[yypt-0].ident if str != "\\" && len(str) > 1 { @@ -20796,7 +20808,7 @@ yynewstate: Value: str, } } - case 2373: + case 2374: { str := yyS[yypt-0].ident if str != "\\" && len(str) > 1 { @@ -20808,118 +20820,118 @@ yynewstate: Value: str, } } - case 2375: + case 2376: { parser.yyVAL.ident = yyS[yypt-0].item.(ast.BinaryLiteral).ToString() } - case 2376: + case 2377: { parser.yyVAL.ident = yyS[yypt-0].item.(ast.BinaryLiteral).ToString() } - case 2377: + case 2378: { parser.yyVAL.item = &ast.LinesClause{Terminated: "\n"} } - case 2378: + case 2379: { parser.yyVAL.item = &ast.LinesClause{Starting: yyS[yypt-1].ident, Terminated: yyS[yypt-0].ident} } - case 2379: + case 2380: { parser.yyVAL.ident = "" } - case 2380: + case 2381: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 2381: + case 2382: { parser.yyVAL.ident = "\n" } - case 2382: + case 2383: { parser.yyVAL.ident = yyS[yypt-0].ident } - case 2383: + case 2384: { parser.yyVAL.item = nil } - case 2384: + case 2385: { parser.yyVAL.item = yyS[yypt-0].item } - case 2385: + case 2386: { l := yyS[yypt-2].item.([]*ast.Assignment) parser.yyVAL.item = append(l, yyS[yypt-0].item.(*ast.Assignment)) } - case 2386: + case 2387: { parser.yyVAL.item = []*ast.Assignment{yyS[yypt-0].item.(*ast.Assignment)} } - case 2387: + case 2388: { parser.yyVAL.item = &ast.Assignment{ Column: yyS[yypt-2].expr.(*ast.ColumnNameExpr).Name, Expr: yyS[yypt-0].expr, } } - case 2388: + case 2389: { parser.yyVAL.statement = &ast.UnlockTablesStmt{} } - case 2389: + case 2390: { parser.yyVAL.statement = &ast.LockTablesStmt{ TableLocks: yyS[yypt-0].item.([]ast.TableLock), } } - case 2392: + case 2393: { parser.yyVAL.item = ast.TableLock{ Table: yyS[yypt-1].item.(*ast.TableName), Type: yyS[yypt-0].item.(model.TableLockType), } } - case 2393: + case 2394: { parser.yyVAL.item = model.TableLockRead } - case 2394: + case 2395: { parser.yyVAL.item = model.TableLockReadLocal } - case 2395: + case 2396: { parser.yyVAL.item = model.TableLockWrite } - case 2396: + case 2397: { parser.yyVAL.item = model.TableLockWriteLocal } - case 2397: + case 2398: { parser.yyVAL.item = []ast.TableLock{yyS[yypt-0].item.(ast.TableLock)} } - case 2398: + case 2399: { parser.yyVAL.item = append(yyS[yypt-2].item.([]ast.TableLock), yyS[yypt-0].item.(ast.TableLock)) } - case 2399: + case 2400: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), TiDBExtension: yyS[yypt-1].item.(bool), } } - case 2400: + case 2401: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), TiDBExtension: yyS[yypt-2].item.(bool), } } - case 2401: + case 2402: { parser.yyVAL.statement = &ast.KillStmt{ ConnectionID: getUint64FromNUM(yyS[yypt-0].item), @@ -20927,28 +20939,28 @@ yynewstate: TiDBExtension: yyS[yypt-2].item.(bool), } } - case 2402: + case 2403: { parser.yyVAL.item = false } - case 2403: + case 2404: { parser.yyVAL.item = true } - case 2404: + case 2405: { parser.yyVAL.statement = &ast.LoadStatsStmt{ Path: yyS[yypt-0].ident, } } - case 2405: + case 2406: { parser.yyVAL.statement = &ast.DropPlacementPolicyStmt{ IfExists: yyS[yypt-1].item.(bool), PolicyName: model.NewCIStr(yyS[yypt-0].ident), } } - case 2406: + case 2407: { parser.yyVAL.statement = &ast.CreatePlacementPolicyStmt{ OrReplace: yyS[yypt-5].item.(bool), @@ -20957,7 +20969,7 @@ yynewstate: PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } - case 2407: + case 2408: { parser.yyVAL.statement = &ast.AlterPlacementPolicyStmt{ IfExists: yyS[yypt-2].item.(bool), @@ -20965,7 +20977,7 @@ yynewstate: PlacementOptions: yyS[yypt-0].item.([]*ast.PlacementOption), } } - case 2408: + case 2409: { parser.yyVAL.statement = &ast.CreateSequenceStmt{ IfNotExists: yyS[yypt-3].item.(bool), @@ -20974,21 +20986,17 @@ yynewstate: TblOptions: yyS[yypt-0].item.([]*ast.TableOption), } } - case 2409: + case 2410: { parser.yyVAL.item = []*ast.SequenceOption{} } - case 2411: - { - parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} - } case 2412: { - parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) + parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} } case 2413: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceOptionIncrementBy, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) } case 2414: { @@ -20996,7 +21004,7 @@ yynewstate: } case 2415: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceStartWith, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceOptionIncrementBy, IntValue: yyS[yypt-0].item.(int64)} } case 2416: { @@ -21004,11 +21012,11 @@ yynewstate: } case 2417: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMinValue, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceStartWith, IntValue: yyS[yypt-0].item.(int64)} } case 2418: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMinValue} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMinValue, IntValue: yyS[yypt-0].item.(int64)} } case 2419: { @@ -21016,11 +21024,11 @@ yynewstate: } case 2420: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMaxValue, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMinValue} } case 2421: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMaxValue} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceMaxValue, IntValue: yyS[yypt-0].item.(int64)} } case 2422: { @@ -21028,11 +21036,11 @@ yynewstate: } case 2423: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCache, IntValue: yyS[yypt-0].item.(int64)} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoMaxValue} } case 2424: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCache} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCache, IntValue: yyS[yypt-0].item.(int64)} } case 2425: { @@ -21040,21 +21048,25 @@ yynewstate: } case 2426: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCycle} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCache} } case 2427: { - parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceCycle} } case 2428: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} } - case 2430: + case 2429: { - parser.yyVAL.item = yyS[yypt-0].item + parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceNoCycle} } case 2431: + { + parser.yyVAL.item = yyS[yypt-0].item + } + case 2432: { unsigned_num := getUint64FromNUM(yyS[yypt-0].item) if unsigned_num > 9223372036854775808 { @@ -21067,14 +21079,14 @@ yynewstate: parser.yyVAL.item = -int64(unsigned_num) } } - case 2432: + case 2433: { parser.yyVAL.statement = &ast.DropSequenceStmt{ IfExists: yyS[yypt-1].item.(bool), Sequences: yyS[yypt-0].item.([]*ast.TableName), } } - case 2433: + case 2434: { parser.yyVAL.statement = &ast.AlterSequenceStmt{ IfExists: yyS[yypt-2].item.(bool), @@ -21082,27 +21094,27 @@ yynewstate: SeqOptions: yyS[yypt-0].item.([]*ast.SequenceOption), } } - case 2434: + case 2435: { parser.yyVAL.item = []*ast.SequenceOption{yyS[yypt-0].item.(*ast.SequenceOption)} } - case 2435: + case 2436: { parser.yyVAL.item = append(yyS[yypt-1].item.([]*ast.SequenceOption), yyS[yypt-0].item.(*ast.SequenceOption)) } - case 2437: + case 2438: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestart} } - case 2438: + case 2439: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestartWith, IntValue: yyS[yypt-0].item.(int64)} } - case 2439: + case 2440: { parser.yyVAL.item = &ast.SequenceOption{Tp: ast.SequenceRestartWith, IntValue: yyS[yypt-0].item.(int64)} } - case 2440: + case 2441: { x := &ast.IndexAdviseStmt{ Path: yyS[yypt-3].ident, @@ -21119,42 +21131,42 @@ yynewstate: } parser.yyVAL.statement = x } - case 2441: + case 2442: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2442: + case 2443: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2443: + case 2444: { parser.yyVAL.item = nil } - case 2444: + case 2445: { parser.yyVAL.item = &ast.MaxIndexNumClause{ PerTable: yyS[yypt-1].item.(uint64), PerDB: yyS[yypt-0].item.(uint64), } } - case 2445: + case 2446: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2446: + case 2447: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2447: + case 2448: { parser.yyVAL.item = uint64(ast.UnspecifiedSize) } - case 2448: + case 2449: { parser.yyVAL.item = getUint64FromNUM(yyS[yypt-0].item) } - case 2449: + case 2450: { // Parse it but will ignore it switch yyS[yypt-0].ident { @@ -21169,19 +21181,19 @@ yynewstate: } parser.yyVAL.ident = yyS[yypt-0].ident } - case 2450: + case 2451: { parser.yyVAL.item = append([]*ast.RowExpr{}, yyS[yypt-0].item.(*ast.RowExpr)) } - case 2451: + case 2452: { parser.yyVAL.item = append(yyS[yypt-2].item.([]*ast.RowExpr), yyS[yypt-0].item.(*ast.RowExpr)) } - case 2452: + case 2453: { parser.yyVAL.item = &ast.RowExpr{Values: yyS[yypt-0].item.([]ast.ExprNode)} } - case 2453: + case 2454: { x := &ast.PlanReplayerStmt{ Stmt: yyS[yypt-0].statement, @@ -21197,7 +21209,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2454: + case 2455: { x := &ast.PlanReplayerStmt{ Stmt: yyS[yypt-0].statement, @@ -21213,7 +21225,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2455: + case 2456: { x := &ast.PlanReplayerStmt{ Stmt: nil, @@ -21233,7 +21245,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2456: + case 2457: { x := &ast.PlanReplayerStmt{ Stmt: nil, @@ -21253,7 +21265,7 @@ yynewstate: parser.yyVAL.statement = x } - case 2457: + case 2458: { x := &ast.PlanReplayerStmt{ Stmt: nil, diff --git a/parser/parser.y b/parser/parser.y index 8a40a7c22de2e..31278b898ed81 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -748,7 +748,6 @@ import ( reset "RESET" regions "REGIONS" region "REGION" - builtinAddDate builtinBitAnd builtinBitOr builtinBitXor @@ -766,7 +765,6 @@ import ( builtinMin builtinNow builtinPosition - builtinSubDate builtinSubstring builtinSum builtinSysDate @@ -3669,6 +3667,17 @@ DatabaseOption: UintValue: placementOptions.UintValue, } } +| "SET" "TIFLASH" "REPLICA" LengthNum LocationLabelList + { + tiflashReplicaSpec := &ast.TiFlashReplicaSpec{ + Count: $4.(uint64), + Labels: $5.([]string), + } + $$ = &ast.DatabaseOption{ + Tp: ast.DatabaseSetTiFlashReplica, + TiFlashReplica: tiflashReplicaSpec, + } + } DatabaseOptionListOpt: { @@ -7235,8 +7244,8 @@ FunctionNameDateArith: | builtinDateSub FunctionNameDateArithMultiForms: - builtinAddDate -| builtinSubDate + addDate +| subDate TrimDirection: "BOTH" diff --git a/parser/parser_test.go b/parser/parser_test.go index 9031d75c3d813..34e26c00117cd 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -2195,6 +2195,111 @@ func TestIdentifier(t *testing.T) { RunTest(t, table, false) } +func TestBuiltinFuncAsIdentifier(t *testing.T) { + whitespaceFuncs := []struct { + funcName string + args string + }{ + {"BIT_AND", "`c1`"}, + {"BIT_OR", "`c1`"}, + {"BIT_XOR", "`c1`"}, + {"CAST", "1 AS FLOAT"}, + {"COUNT", "1"}, + {"CURDATE", ""}, + {"CURTIME", ""}, + {"DATE_ADD", "_UTF8MB4'2011-11-11 10:10:10', INTERVAL 10 SECOND"}, + {"DATE_SUB", "_UTF8MB4'2011-11-11 10:10:10', INTERVAL 10 SECOND"}, + {"EXTRACT", "SECOND FROM _UTF8MB4'2011-11-11 10:10:10'"}, + {"GROUP_CONCAT", "`c2`, `c1` SEPARATOR ','"}, + {"MAX", "`c1`"}, + {"MID", "_UTF8MB4'Sakila', -5, 3"}, + {"MIN", "`c1`"}, + {"NOW", ""}, + {"POSITION", "_UTF8MB4'bar' IN _UTF8MB4'foobarbar'"}, + {"STDDEV_POP", "`c1`"}, + {"STDDEV_SAMP", "`c1`"}, + {"SUBSTR", "_UTF8MB4'Quadratically', 5"}, + {"SUBSTRING", "_UTF8MB4'Quadratically', 5"}, + {"SUM", "`c1`"}, + {"SYSDATE", ""}, + {"TRIM", "_UTF8MB4' foo '"}, + {"VAR_POP", "`c1`"}, + {"VAR_SAMP", "`c1`"}, + } + + testcases := make([]testCase, 0, 3*len(whitespaceFuncs)) + runTests := func(ignoreSpace bool) { + p := parser.New() + if ignoreSpace { + p.SetSQLMode(mysql.ModeIgnoreSpace) + } + for _, c := range testcases { + _, _, err := p.Parse(c.src, "", "") + if !c.ok { + require.Errorf(t, err, "source %v", c.src) + continue + } + require.NoErrorf(t, err, "source %v", c.src) + if c.ok && !ignoreSpace { + RunRestoreTest(t, c.src, c.restore, false) + } + } + } + + for _, function := range whitespaceFuncs { + // `x` is recognized as a function name for `x()`. + testcases = append(testcases, testCase{fmt.Sprintf("select %s(%s)", function.funcName, function.args), true, fmt.Sprintf("SELECT %s(%s)", function.funcName, function.args)}) + + // In MySQL, `select x ()` is recognized as a stored function. + // In TiDB, most of these functions are recognized as identifiers while some are builtin functions (such as COUNT, CURDATE) + // because the later ones are not added to the token map. We'd better not to modify it since it breaks compatibility. + // For example, `select CURDATE ()` reports an error in MySQL but it works well for TiDB. + + // `x` is recognized as an identifier for `x ()`. + testcases = append(testcases, testCase{fmt.Sprintf("create table %s (a int)", function.funcName), true, fmt.Sprintf("CREATE TABLE `%s` (`a` INT)", function.funcName)}) + + // `x` is recognized as a function name for `x()`. + testcases = append(testcases, testCase{fmt.Sprintf("create table %s(a int)", function.funcName), false, ""}) + } + runTests(false) + + testcases = make([]testCase, 0, 4*len(whitespaceFuncs)) + for _, function := range whitespaceFuncs { + testcases = append(testcases, testCase{fmt.Sprintf("select %s(%s)", function.funcName, function.args), true, fmt.Sprintf("SELECT %s(%s)", function.funcName, function.args)}) + testcases = append(testcases, testCase{fmt.Sprintf("select %s (%s)", function.funcName, function.args), true, fmt.Sprintf("SELECT %s(%s)", function.funcName, function.args)}) + testcases = append(testcases, testCase{fmt.Sprintf("create table %s (a int)", function.funcName), false, ""}) + testcases = append(testcases, testCase{fmt.Sprintf("create table %s(a int)", function.funcName), false, ""}) + } + runTests(true) + + normalFuncs := []struct { + funcName string + args string + }{ + {"ADDDATE", "_UTF8MB4'2011-11-11 10:10:10', INTERVAL 10 SECOND"}, + {"SESSION_USER", ""}, + {"SUBDATE", "_UTF8MB4'2011-11-11 10:10:10', INTERVAL 10 SECOND"}, + {"SYSTEM_USER", ""}, + } + + testcases = make([]testCase, 0, 4*len(normalFuncs)) + for _, function := range normalFuncs { + // `x` is recognized as a function name for `select x()`. + testcases = append(testcases, testCase{fmt.Sprintf("select %s(%s)", function.funcName, function.args), true, fmt.Sprintf("SELECT %s(%s)", function.funcName, function.args)}) + + // `x` is recognized as a function name for `select x ()`. + testcases = append(testcases, testCase{fmt.Sprintf("select %s (%s)", function.funcName, function.args), true, fmt.Sprintf("SELECT %s(%s)", function.funcName, function.args)}) + + // `x` is recognized as an identifier for `create table x ()`. + testcases = append(testcases, testCase{fmt.Sprintf("create table %s (a int)", function.funcName), true, fmt.Sprintf("CREATE TABLE `%s` (`a` INT)", function.funcName)}) + + // `x` is recognized as an identifier for `create table x()`. + testcases = append(testcases, testCase{fmt.Sprintf("create table %s(a int)", function.funcName), true, fmt.Sprintf("CREATE TABLE `%s` (`a` INT)", function.funcName)}) + } + runTests(false) + runTests(true) +} + func TestDDL(t *testing.T) { table := []testCase{ {"CREATE", false, ""}, @@ -3235,6 +3340,12 @@ func TestDDL(t *testing.T) { {"ALTER TABLE t SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", true, "ALTER TABLE `t` SET TIFLASH REPLICA 2 LOCATION LABELS 'a', 'b'"}, {"ALTER TABLE t SET TIFLASH REPLICA 0", true, "ALTER TABLE `t` SET TIFLASH REPLICA 0"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 2 LOCATION LABELS 'a', 'b'"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 0", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 0"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2 LOCATION LABELS 'a', 'b'"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 1 LOCATION LABELS 'a', 'b' SET TIFLASH REPLICA 2"}, + {"ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2 LOCATION LABELS 'a', 'b'", true, "ALTER DATABASE `t` SET TIFLASH REPLICA 1 LOCATION LABELS 'a', 'b' SET TIFLASH REPLICA 2 LOCATION LABELS 'a', 'b'"}, // for issue 537 {"CREATE TABLE IF NOT EXISTS table_ident (a SQL_TSI_YEAR(4), b SQL_TSI_YEAR);", true, "CREATE TABLE IF NOT EXISTS `table_ident` (`a` YEAR(4),`b` YEAR)"}, diff --git a/planner/cascades/main_test.go b/planner/cascades/main_test.go index f1a3ceb81fe23..1951bb916a2c6 100644 --- a/planner/cascades/main_test.go +++ b/planner/cascades/main_test.go @@ -47,7 +47,7 @@ func TestMain(m *testing.M) { testDataMap.GenerateOutputIfNeeded() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/planner/cascades/testdata/transformation_rules_suite_out.json b/planner/cascades/testdata/transformation_rules_suite_out.json index 185e2441edd11..13a9b993d5da2 100644 --- a/planner/cascades/testdata/transformation_rules_suite_out.json +++ b/planner/cascades/testdata/transformation_rules_suite_out.json @@ -353,10 +353,14 @@ " Projection_3 input:[Group#1], test.t.b, Column#13", "Group#1 Schema:[Column#13,test.t.b]", " Aggregation_2 input:[Group#2], group by:plus(sin(cast(test.t.b, double BINARY)), sin(cast(test.t.c, double BINARY))), test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)", + " Aggregation_7 input:[Group#3], group by:Column#14, test.t.b, funcs:sum(Column#15), firstrow(test.t.b)", "Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TiKVSingleGather_5 input:[Group#3], table:t", - "Group#3 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", - " TableScan_4 table:t, pk col:test.t.a" + " TiKVSingleGather_5 input:[Group#4], table:t", + "Group#4 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]", + " TableScan_4 table:t, pk col:test.t.a", "Group#3 Schema:[Column#15,Column#14,test.t.b]", + " TiKVSingleGather_5 input:[Group#5], table:t", + "Group#5 Schema:[Column#15,Column#14,test.t.b]", + " Aggregation_6 input:[Group#4], group by:plus(sin(cast(test.t.b, double BINARY)), sin(cast(test.t.c, double BINARY))), test.t.b, funcs:sum(test.t.a)" ] } ] diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 6a0d12eb977c0..8cddc669be271 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -442,7 +442,7 @@ func (r *PushAggDownGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.Gr gbyItems := make([]expression.Expression, len(agg.GroupByItems)) copy(gbyItems, agg.GroupByItems) - partialPref, finalPref, funcMap := plannercore.BuildFinalModeAggregation(agg.SCtx(), + partialPref, finalPref, firstRowFuncMap := plannercore.BuildFinalModeAggregation(agg.SCtx(), &plannercore.AggInfo{ AggFuncs: aggFuncs, GroupByItems: gbyItems, @@ -453,7 +453,7 @@ func (r *PushAggDownGather) OnTransform(old *memo.ExprIter) (newExprs []*memo.Gr } // Remove unnecessary FirstRow. partialPref.AggFuncs = - plannercore.RemoveUnnecessaryFirstRow(agg.SCtx(), finalPref.AggFuncs, finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, funcMap) + plannercore.RemoveUnnecessaryFirstRow(agg.SCtx(), finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, firstRowFuncMap) partialAgg := plannercore.LogicalAggregation{ AggFuncs: partialPref.AggFuncs, diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 603dfd030611a..cdcffdb0addd0 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -23,43 +23,21 @@ import ( "strings" "testing" - . "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testAnalyzeSuite{}) - -type testAnalyzeSuite struct { - testData testutil.TestData -} - -func (s *testAnalyzeSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "analyze_suite") - c.Assert(err, IsNil) -} - -func (s *testAnalyzeSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) error { +func loadTableStats(fileName string, dom *domain.Domain) error { statsPath := filepath.Join("testdata", fileName) bytes, err := os.ReadFile(statsPath) if err != nil { @@ -78,15 +56,10 @@ func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) e return nil } -func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestExplainAnalyze(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by tk.MustExec("create table t1(a int, b int, c int, key idx(a, b))") @@ -95,151 +68,134 @@ func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) { tk.MustExec("insert into t2 values (2, 22), (3, 33), (5, 55), (233, 2), (333, 3), (3434, 5)") tk.MustExec("analyze table t1, t2") rs := tk.MustQuery("explain analyze select t1.a, t1.b, sum(t1.c) from t1 join t2 on t1.a = t2.b where t1.a > 1") - c.Assert(len(rs.Rows()), Equals, 10) + require.Len(t, rs.Rows(), 10) for _, row := range rs.Rows() { - c.Assert(len(row), Equals, 9) + require.Len(t, row, 9) execInfo := row[5].(string) - c.Assert(strings.Contains(execInfo, "time"), Equals, true) - c.Assert(strings.Contains(execInfo, "loops"), Equals, true) + require.Contains(t, execInfo, "time") + require.Contains(t, execInfo, "loops") if strings.Contains(row[0].(string), "Reader") || strings.Contains(row[0].(string), "IndexLookUp") { - c.Assert(strings.Contains(execInfo, "cop_task"), Equals, true) + require.Contains(t, execInfo, "cop_task") } } } // TestCBOWithoutAnalyze tests the plan with stats that only have count info. -func (s *testAnalyzeSuite) TestCBOWithoutAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCBOWithoutAnalyze(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t1 (a int)") testKit.MustExec("create table t2 (a int)") h := dom.StatsHandle() - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) testKit.MustExec("insert into t1 values (1), (2), (3), (4), (5), (6)") testKit.MustExec("insert into t2 values (1), (2), (3), (4), (5), (6)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestStraightJoin(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestStraightJoin(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") h := dom.StatsHandle() for _, tblName := range []string{"t1", "t2", "t3", "t4"} { testKit.MustExec(fmt.Sprintf("create table %s (a int)", tblName)) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) } var input []string var output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestTableDual(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() +func TestTableDual(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() - testKit := testkit.NewTestKit(c, store) + testKit := testkit.NewTestKit(t, store) testKit.MustExec(`use test`) h := dom.StatsHandle() testKit.MustExec(`create table t(a int)`) testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - statistics.RatioOfPseudoEstimate.Store(0.7) - }() +func TestEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) statistics.RatioOfPseudoEstimate.Store(10.0) + defer statistics.RatioOfPseudoEstimate.Store(0.7) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") h := dom.StatsHandle() - err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") for i := 1; i <= 8; i++ { testKit.MustExec("delete from t where a = ?", i) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := testKit.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } @@ -256,15 +212,10 @@ func constructInsertSQL(i, n int) string { return sql } -func (s *testAnalyzeSuite) TestIndexRead(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIndexRead(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("set @@session.tidb_executor_concurrency = 4;") testKit.MustExec("set @@session.tidb_hash_join_concurrency = 5;") testKit.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") @@ -281,78 +232,69 @@ func (s *testAnalyzeSuite) TestIndexRead(c *C) { // This stats is generated by following format: // fill (a, b, c, e) as (i*100+j, i, i+j, i*100+j), i and j is dependent and range of this two are [0, 99]. - err = s.loadTableStats("analyzesSuiteTestIndexReadT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzesSuiteTestIndexReadT.json", dom)) for i := 1; i < 16; i++ { testKit.MustExec(fmt.Sprintf("insert into t1 values(%v, %v)", i, i)) } testKit.MustExec("analyze table t1") - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestEmptyTable(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestEmptyTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1") testKit.MustExec("create table t (c1 int)") testKit.MustExec("create table t1 (c1 int)") testKit.MustExec("analyze table t, t1") var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestAnalyze(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1, t2, t3") testKit.MustExec("create table t (a int, b int)") @@ -383,63 +325,58 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { testKit.MustExec("analyze table t4") testKit.MustExec("create view v as select * from t") - _, err = testKit.Exec("analyze table v") - c.Assert(err.Error(), Equals, "analyze view v is not supported now.") + _, err := testKit.Exec("analyze table v") + require.EqualError(t, err, "analyze view v is not supported now.") testKit.MustExec("drop view v") testKit.MustExec("create sequence seq") _, err = testKit.Exec("analyze table seq") - c.Assert(err.Error(), Equals, "analyze sequence seq is not supported now.") + require.EqualError(t, err, "analyze sequence seq is not supported now.") testKit.MustExec("drop sequence seq") var input, output []string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] err = executor.ResetContextOfStmt(ctx, stmt) - c.Assert(err, IsNil) + require.NoError(t, err) ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i] = planString }) - c.Assert(planString, Equals, output[i], Commentf("for %s", tt)) + require.Equalf(t, output[i], planString, "case: %v", tt) } } -func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestOutdatedAnalyze(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx(a))") for i := 0; i < 10; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d,%d)", i, i)) } h := dom.StatsHandle() - err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") testKit.MustExec("insert into t select * from t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(dom.InfoSchema())) var input []struct { SQL string EnablePseudoForOutdatedStats bool @@ -451,70 +388,26 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { RatioOfPseudoEstimate float64 Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - testKit.Se.GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) + testKit.Session().GetSessionVars().SetEnablePseudoForOutdatedStats(tt.EnablePseudoForOutdatedStats) statistics.RatioOfPseudoEstimate.Store(tt.RatioOfPseudoEstimate) plan := testKit.MustQuery(tt.SQL) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt.SQL output[i].EnablePseudoForOutdatedStats = tt.EnablePseudoForOutdatedStats output[i].RatioOfPseudoEstimate = tt.RatioOfPseudoEstimate - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestPreparedNullParam(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - - defer config.RestoreFunc()() - flags := []bool{false, true} - for _, flag := range flags { - config.UpdateGlobal(func(conf *config.Config) { - conf.PreparedPlanCache.Enabled = flag - conf.PreparedPlanCache.Capacity = 100 - }) - testKit := testkit.NewTestKit(c, store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t (id int not null, KEY id (id))") - testKit.MustExec("insert into t values (1), (2), (3)") - - sql := "select * from t where id = ?" - best := "Dual" - - ctx := testKit.Se.(sessionctx.Context) - stmts, err := session.Parse(ctx, sql) - c.Assert(err, IsNil) - stmt := stmts[0] - - ret := &core.PreprocessorReturn{} - err = core.Preprocess(ctx, stmt, core.InPrepare, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) - - c.Assert(core.ToString(p), Equals, best, Commentf("for %s", sql)) - } -} - -func (s *testAnalyzeSuite) TestNullCount(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestNullCount(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int, b int, index idx(a))") @@ -522,33 +415,29 @@ func (s *testAnalyzeSuite) TestNullCount(c *C) { testKit.MustExec("analyze table t") var input []string var output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i := 0; i < 2; i++ { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) }) testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) } h := dom.StatsHandle() h.Clear() - c.Assert(h.Update(dom.InfoSchema()), IsNil) + require.NoError(t, h.Update(dom.InfoSchema())) for i := 2; i < 4; i++ { - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) }) testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCorrelatedEstimation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by tk.MustExec("create table t(a int, b int, c int, index idx(c,b,a))") @@ -558,25 +447,21 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { input []string output [][]string ) - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { rs := tk.MustQuery(tt) - s.testData.OnRecord(func() { - output[i] = s.testData.ConvertRowsToStrings(rs.Rows()) + testdata.OnRecord(func() { + output[i] = testdata.ConvertRowsToStrings(rs.Rows()) }) rs.Check(testkit.Rows(output[i]...)) } } -func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestInconsistentEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int, index ab(a,b), index ac(a,c))") tk.MustExec("insert into t values (1,1,1), (1000,1000,1000)") @@ -588,52 +473,29 @@ func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { // Force using the histogram to estimate. tk.MustExec("update mysql.stats_histograms set stats_ver = 0") dom.StatsHandle().Clear() - err = dom.StatsHandle().Update(dom.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, sql := range input { plan := tk.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) } } -func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { - store, err := mockstore.NewMockStore() - if err != nil { - return nil, nil, errors.Trace(err) - } - - session.SetSchemaLease(0) - session.DisableStats4Test() - - dom, err := session.BootstrapSession(store) - if err != nil { - return nil, nil, err - } - - dom.SetStatsUpdating(true) - return store, dom, errors.Trace(err) -} - func BenchmarkOptimize(b *testing.B) { - c := &C{} - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() + store, clean := testkit.CreateMockStore(b) + defer clean() - testKit := testkit.NewTestKit(c, store) + testKit := testkit.NewTestKit(b, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int primary key, b int, c varchar(200), d datetime DEFAULT CURRENT_TIMESTAMP, e int, ts timestamp DEFAULT CURRENT_TIMESTAMP)") @@ -727,35 +589,30 @@ func BenchmarkOptimize(b *testing.B) { }, } for _, tt := range tests { - ctx := testKit.Se.(sessionctx.Context) + ctx := testKit.Session() stmts, err := session.Parse(ctx, tt.sql) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(b, err) + require.Len(b, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(b, err) b.Run(tt.sql, func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { _, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(b, err) } b.ReportAllocs() }) } } -func (s *testAnalyzeSuite) TestIssue9562(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIssue9562(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") var input [][]string @@ -763,16 +620,17 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { SQL []string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -782,15 +640,10 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { } } -func (s *testAnalyzeSuite) TestIssue9805(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIssue9805(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec(` @@ -816,15 +669,10 @@ func (s *testAnalyzeSuite) TestIssue9805(c *C) { tk.MustQuery("explain analyze select /*+ TIDB_INLJ(t2) */ t1.id, t2.a from t1 join t2 on t1.a = t2.d where t1.b = 't2' and t1.d = 4") } -func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLimitCrossEstimation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@session.tidb_executor_concurrency = 4;") tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") @@ -837,16 +685,17 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { SQL []string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -856,21 +705,15 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { } } -func (s *testAnalyzeSuite) TestLowSelIndexGreedySearch(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLowSelIndexGreedySearch(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec(`set tidb_opt_limit_push_down_threshold=0`) testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a varchar(32) default null, b varchar(10) default null, c varchar(12) default null, d varchar(32) default null, e bigint(10) default null, key idx1 (d,a), key idx2 (a,c), key idx3 (c,b), key idx4 (e))") - err = s.loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestLowSelIndexGreedySearchT.json", dom)) var input []string var output []struct { SQL string @@ -879,59 +722,52 @@ func (s *testAnalyzeSuite) TestLowSelIndexGreedySearch(c *C) { // The test purposes are: // - index `idx2` runs much faster than `idx4` experimentally; // - estimated row count of IndexLookUp should be 0; - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestUpdateProjEliminate(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestUpdateProjEliminate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("explain update t t1, (select distinct b from t) t2 set t1.b = t2.b") } -func (s *testAnalyzeSuite) TestTiFlashCostModel(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestTiFlashCostModel(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int, b int, c int, primary key(a))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t", L: "t"}) - c.Assert(err, IsNil) + require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} var input, output [][]string - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { if j == len(ts)-1 { - output[i] = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i] = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -941,45 +777,35 @@ func (s *testAnalyzeSuite) TestTiFlashCostModel(c *C) { } } -func (s *testAnalyzeSuite) TestIndexEqualUnknown(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIndexEqualUnknown(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1") - testKit.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly testKit.MustExec("CREATE TABLE t(a bigint(20) NOT NULL, b bigint(20) NOT NULL, c bigint(20) NOT NULL, PRIMARY KEY (a,c,b), KEY (b))") - err = s.loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestIndexEqualUnknownT.json", dom)) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(testKit.MustQuery(tt).Rows()) }) testKit.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestLimitIndexEstimation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestLimitIndexEstimation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -987,31 +813,27 @@ func (s *testAnalyzeSuite) TestLimitIndexEstimation(c *C) { tk.MustExec("set session tidb_enable_extended_stats = on") // Values in column a are from 1 to 1000000, values in column b are from 1000000 to 1, // these 2 columns are strictly correlated in reverse order. - err = s.loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom) - c.Assert(err, IsNil) + require.NoError(t, loadTableStats("analyzeSuiteTestLimitIndexEstimationT.json", dom)) var input []string var output []struct { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + analyzeSuiteData := core.GetAnalyzeSuiteData() + analyzeSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - testKit := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestBatchPointGetTablePartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t1,t2,t3,t4,t5,t6") @@ -1092,7 +914,7 @@ func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { "1 2", )) - testKit.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn testKit.MustExec("create table t3(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") testKit.MustExec("insert into t3 values(1,1),(1,2),(2,1),(2,2)") @@ -1250,14 +1072,10 @@ func (s *testAnalyzeSuite) TestBatchPointGetTablePartition(c *C) { } // TestAppendIntPkToIndexTailForRangeBuilding tests for issue25219 https://github.com/pingcap/tidb/issues/25219. -func (s *testAnalyzeSuite) TestAppendIntPkToIndexTailForRangeBuilding(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestAppendIntPkToIndexTailForRangeBuilding(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t25219(a int primary key, col3 int, col1 int, index idx(col3))") tk.MustExec("insert into t25219 values(1, 1, 1)") diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 9127c9a2335a2..cab9b46085a7b 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" @@ -41,6 +42,7 @@ import ( "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/kvcache" @@ -427,20 +429,6 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx prepared := preparedStmt.PreparedAst - if prepared.UseCache { - // disable the cache if cache table in prepared statement - for _, vInfo := range preparedStmt.VisitInfos { - tbl, err := is.TableByName(model.NewCIStr(vInfo.db), model.NewCIStr(vInfo.table)) - // if table does not exist, skip it, maybe it is a `create table` statement - if err != nil { - continue - } - if tbl.Meta().TableCacheStatusType == model.TableCacheStatusEnable { - prepared.UseCache = false - break - } - } - } stmtCtx.UseCache = prepared.UseCache var bindSQL string @@ -720,18 +708,25 @@ func (e *Execute) rebuildRange(p Plan) error { // TODO: relocate the partition after rebuilding range to make PlanCache support PointGet return errors.New("point get for partition table can not use plan cache") } - if x.HandleParam != nil { - var iv int64 - iv, err = x.HandleParam.Datum.ToInt64(sc) + if x.HandleConstant != nil { + dVal, err := convertConstant2Datum(sc, x.HandleConstant, x.handleFieldType) + if err != nil { + return err + } + iv, err := dVal.ToInt64(sc) if err != nil { return err } x.Handle = kv.IntHandle(iv) return nil } - for i, param := range x.IndexValueParams { + for i, param := range x.IndexConstants { if param != nil { - x.IndexValues[i] = param.Datum + dVal, err := convertConstant2Datum(sc, param, x.ColsFieldType[i]) + if err != nil { + return err + } + x.IndexValues[i] = *dVal } } return nil @@ -825,6 +820,23 @@ func (e *Execute) rebuildRange(p Plan) error { return nil } +func convertConstant2Datum(sc *stmtctx.StatementContext, con *expression.Constant, target *types.FieldType) (*types.Datum, error) { + val, err := con.Eval(chunk.Row{}) + if err != nil { + return nil, err + } + dVal, err := val.ConvertTo(sc, target) + if err != nil { + return nil, err + } + // The converted result must be same as original datum. + cmp, err := dVal.Compare(sc, &val, collate.GetCollator(target.Collate)) + if err != nil || cmp != 0 { + return nil, errors.New("Convert constant to datum is failed, because the constant has changed after the covert") + } + return &dVal, nil +} + func (e *Execute) buildRangeForTableScan(sctx sessionctx.Context, ts *PhysicalTableScan) (err error) { if ts.Table.IsCommonHandle { pk := tables.FindPrimaryIndex(ts.Table) diff --git a/planner/core/enforce_mpp_test.go b/planner/core/enforce_mpp_test.go index f9eb265c1313f..8cc599d075d28 100644 --- a/planner/core/enforce_mpp_test.go +++ b/planner/core/enforce_mpp_test.go @@ -16,49 +16,22 @@ package core_test import ( "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testEnforceMPPSuite{}) - -type testEnforceMPPSuite struct { - testData testutil.TestData - store kv.Storage - dom *domain.Domain -} - -func (s *testEnforceMPPSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "enforce_mpp_suite") - c.Assert(err, IsNil) -} - -func (s *testEnforceMPPSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testEnforceMPPSuite) SetUpTest(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testEnforceMPPSuite) TearDownTest(c *C) { - s.dom.Close() - err := s.store.Close() - c.Assert(err, IsNil) -} - -func (s *testEnforceMPPSuite) TestSetVariables(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSetVariables(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test value limit of tidb_opt_tiflash_concurrency_factor tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 0") @@ -67,18 +40,17 @@ func (s *testEnforceMPPSuite) TestSetVariables(c *C) { // test set tidb_enforce_mpp when tidb_allow_mpp=false; err := tk.ExecToErr("set @@tidb_allow_mpp = 0; set @@tidb_enforce_mpp = 1;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_enforce_mpp' can't be set to the value of '1' but tidb_allow_mpp is 0, please activate tidb_allow_mpp at first.'`) - + require.EqualError(t, err, `[variable:1231]Variable 'tidb_enforce_mpp' can't be set to the value of '1' but tidb_allow_mpp is 0, please activate tidb_allow_mpp at first.'`) err = tk.ExecToErr("set @@tidb_allow_mpp = 1; set @@tidb_enforce_mpp = 1;") - c.Assert(err, IsNil) - + require.NoError(t, err) err = tk.ExecToErr("set @@tidb_allow_mpp = 0;") - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testEnforceMPPSuite) TestEnforceMPP(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnforceMPP(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test query tk.MustExec("use test") @@ -87,10 +59,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPP(c *C) { tk.MustExec("create index idx on t(a)") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -106,7 +78,8 @@ func (s *testEnforceMPPSuite) TestEnforceMPP(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) filterWarnings := func(originalWarnings []stmtctx.SQLWarn) []stmtctx.SQLWarn { warnings := make([]stmtctx.SQLWarn, 0, 4) for _, warning := range originalWarnings { @@ -118,27 +91,29 @@ func (s *testEnforceMPPSuite) TestEnforceMPP(c *C) { return warnings } for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") { tk.MustExec(tt) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(filterWarnings(tk.Se.GetSessionVars().StmtCtx.GetWarnings())) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(filterWarnings(tk.Se.GetSessionVars().StmtCtx.GetWarnings())), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(filterWarnings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()))) } } // general cases. -func (s *testEnforceMPPSuite) TestEnforceMPPWarning1(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnforceMPPWarning1(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test query tk.MustExec("use test") @@ -152,9 +127,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning1(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") { @@ -163,10 +139,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning1(c *C) { } if strings.HasPrefix(tt, "cmd: create-replica") { // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -179,10 +155,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning1(c *C) { } if strings.HasPrefix(tt, "cmd: enable-replica") { // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -193,20 +169,22 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning1(c *C) { } continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } // partition table. -func (s *testEnforceMPPSuite) TestEnforceMPPWarning2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnforceMPPWarning2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test query tk.MustExec("use test") @@ -214,10 +192,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning2(c *C) { tk.MustExec("CREATE TABLE t (a int, b char(20)) PARTITION BY HASH(a)") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -233,29 +211,32 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning2(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") { tk.MustExec(tt) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } // new collation. -func (s *testEnforceMPPSuite) TestEnforceMPPWarning3(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnforceMPPWarning3(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test query tk.MustExec("use test") @@ -263,10 +244,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning3(c *C) { tk.MustExec("CREATE TABLE t (a int, b char(20))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -282,9 +263,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning3(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { @@ -299,20 +281,23 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning3(c *C) { collate.SetNewCollationEnabledForTest(false) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } + collate.SetNewCollationEnabledForTest(true) } // Test enforce mpp warning for joins -func (s *testEnforceMPPSuite) TestEnforceMPPWarning4(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestEnforceMPPWarning4(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) // test table tk.MustExec("use test") @@ -322,10 +307,10 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning4(c *C) { tk.MustExec("CREATE TABLE s(a int primary key)") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" || tblInfo.Name.L == "s" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -341,22 +326,23 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning4(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { tk.MustExec(tt) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } diff --git a/planner/core/errors.go b/planner/core/errors.go index 6670d64d0b1fd..7182702e9d06a 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -108,4 +108,5 @@ var ( ErrPartitionNoTemporary = dbterror.ClassOptimizer.NewStd(mysql.ErrPartitionNoTemporary) ErrViewSelectTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrViewSelectTmptable) ErrSubqueryMoreThan1Row = dbterror.ClassOptimizer.NewStd(mysql.ErrSubqueryNo1Row) + ErrKeyPart0 = dbterror.ClassOptimizer.NewStd(mysql.ErrKeyPart0) ) diff --git a/planner/core/errors_test.go b/planner/core/errors_test.go index b5afcbc616cb5..0eb26d2ed2702 100644 --- a/planner/core/errors_test.go +++ b/planner/core/errors_test.go @@ -15,16 +15,14 @@ package core import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/stretchr/testify/require" ) -type testErrorSuite struct{} - -var _ = Suite(testErrorSuite{}) - -func (s testErrorSuite) TestError(c *C) { +func TestError(t *testing.T) { kvErrs := []*terror.Error{ ErrUnsupportedType, ErrAnalyzeMissIndex, @@ -84,9 +82,10 @@ func (s testErrorSuite) TestError(c *C) { ErrCartesianProductUnsupported, ErrStmtNotFound, ErrAmbiguous, + ErrKeyPart0, } for _, err := range kvErrs { code := terror.ToSQLError(err).Code - c.Assert(code != mysql.ErrUnknown && code == uint16(err.Code()), IsTrue, Commentf("err: %v", err)) + require.Truef(t, code != mysql.ErrUnknown && code == uint16(err.Code()), "err: %v", err) } } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index db9c16e722d78..3fe30d6b52c50 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -1870,7 +1871,12 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC return nil } - if p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin { + if !expression.IsPushDownEnabled(p.JoinType.String(), kv.TiFlash) { + p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because join type `" + p.JoinType.String() + "` is blocked by blacklist, check `table mysql.expr_pushdown_blacklist;` for more information.") + return nil + } + + if p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin && p.JoinType != LeftOuterSemiJoin && p.JoinType != AntiLeftOuterSemiJoin { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because join type `" + p.JoinType.String() + "` is not supported now.") return nil } @@ -1918,13 +1924,13 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { preferredBuildIndex = 1 } - } else if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { + } else if p.JoinType.IsSemiJoin() { preferredBuildIndex = 1 } if p.JoinType == LeftOuterJoin || p.JoinType == RightOuterJoin { - // TiFlash does not requires that the build side must be the inner table for outer join - // so we can choose the build side based on the row count, except that - // 1. it is a broadcast join(for broadcast join, it make sense to use the broadcast side as the build side) + // TiFlash does not require that the build side must be the inner table for outer join. + // so we can choose the build side based on the row count, except that: + // 1. it is a broadcast join(for broadcast join, it makes sense to use the broadcast side as the build side) // 2. or session variable MPPOuterJoinFixedBuildSide is set to true // 3. or there are otherConditions for this join if useBCJ || p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide || len(p.OtherConditions) > 0 { @@ -1996,6 +2002,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC mppShuffleJoin: !useBCJ, // Mpp Join has quite heavy cost. Even limit might not suspend it in time, so we dont scale the count. }.Init(p.ctx, p.stats, p.blockOffset, childrenProps...) + join.SetSchema(p.schema) return []PhysicalPlan{join} } @@ -2349,6 +2356,13 @@ func (p *baseLogicalPlan) canPushToCopImpl(storeTp kv.StoreType, considerDual bo if (isTopN || isLimit) && considerIndexMerge { return false // TopN and Limit cannot be pushed down to IndexMerge } + if c.tableInfo.TableCacheStatusType != model.TableCacheStatusDisable { + // Don't push to cop for cached table, it brings more harm than good: + // 1. Those tables are small enough, push to cop can't utilize several TiKV to accelerate computation. + // 2. Cached table use UnionScan to read the cache data, and push to cop is not supported when an UnionScan exists. + // Once aggregation is pushed to cop, the cache data can't be use any more. + return false + } case *LogicalUnionAll: if storeTp == kv.TiFlash { ret = ret && c.canPushToCopImpl(storeTp, true) @@ -2492,6 +2506,9 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P } if !la.canPushToCop(kv.TiKV) { taskTypes = []property.TaskType{property.RootTaskType} + if la.canPushToCop(kv.TiFlash) { + taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) + } } for _, taskTp := range taskTypes { copiedChildProperty := new(property.PhysicalProperty) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 217deb9c18c11..475bc4cb60245 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1075,7 +1075,7 @@ func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok er.ctxStackAppend(value, types.EmptyName) case *driver.ParamMarkerExpr: var value expression.Expression - value, er.err = expression.ParamMarkerExpression(er.sctx, v) + value, er.err = expression.ParamMarkerExpression(er.sctx, v, false) if er.err != nil { return retNode, false } @@ -1231,7 +1231,7 @@ func (er *expressionRewriter) newFunction(funcName string, retType *types.FieldT } func (er *expressionRewriter) checkTimePrecision(ft *types.FieldType) error { - if ft.EvalType() == types.ETDuration && ft.Decimal > int(types.MaxFsp) { + if ft.EvalType() == types.ETDuration && ft.Decimal > types.MaxFsp { return errTooBigPrecision.GenWithStackByArgs(ft.Decimal, "CAST", types.MaxFsp) } return nil @@ -2015,7 +2015,7 @@ func (er *expressionRewriter) evalDefaultExpr(v *ast.DefaultExpr) { var val *expression.Constant switch { case isCurrentTimestamp && (col.Tp == mysql.TypeDatetime || col.Tp == mysql.TypeTimestamp): - t, err := expression.GetTimeValue(er.sctx, ast.CurrentTimestamp, col.Tp, int8(col.Decimal)) + t, err := expression.GetTimeValue(er.sctx, ast.CurrentTimestamp, col.Tp, col.Decimal) if err != nil { return } diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index ba8f01cfb7431..ee5f20d3f824c 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -15,53 +15,29 @@ package core_test import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testExpressionRewriterSuite{}) -var _ = SerialSuites(&testExpressionRewriterSuiteSerial{}) - -type testExpressionRewriterSuite struct { - testData testutil.TestData -} - -func (s *testExpressionRewriterSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "expression_rewriter_suite") - c.Assert(err, IsNil) -} - -func (s *testExpressionRewriterSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -type testExpressionRewriterSuiteSerial struct { -} - -func (s *testExpressionRewriterSuite) TestIfNullEliminateColName(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIfNullEliminateColName(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int not null, b int not null)") rs, err := tk.Exec("select ifnull(a,b) from t") - c.Assert(err, IsNil) + require.NoError(t, err) fields := rs.Fields() - c.Assert(fields[0].Column.Name.L, Equals, "ifnull(a,b)") - c.Assert(rs.Close(), IsNil) + require.Greater(t, len(fields), 0) + require.Equal(t, "ifnull(a,b)", rs.Fields()[0].Column.Name.L) + require.NoError(t, rs.Close()) tk.MustExec("drop table if exists t") tk.MustExec("create table t(e int not null, b int)") @@ -74,15 +50,10 @@ func (s *testExpressionRewriterSuite) TestIfNullEliminateColName(c *C) { rows.Check(testkit.Rows("1")) } -func (s *testExpressionRewriterSuite) TestBinaryOpFunction(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestBinaryOpFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE t(a int, b int, c int);") @@ -91,15 +62,10 @@ func (s *testExpressionRewriterSuite) TestBinaryOpFunction(c *C) { tk.MustQuery("SELECT * FROM t WHERE (a,b,c) > (1,2,3) order by b").Check(testkit.Rows("1 3 ")) } -func (s *testExpressionRewriterSuite) TestDefaultFunction(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestDefaultFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1( @@ -121,18 +87,16 @@ func (s *testExpressionRewriterSuite) TestDefaultFunction(c *C) { default(d) as defd, default(e) as defe, default(f) as deff - from t1`).Check(testutil.RowsWithSep("|", "def||10|3.14|2018-01-01 00:00:00|2011-11-11 11:11:11")) - err = tk.ExecToErr("select default(x) from t1") - c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'x' in 'field list'") + from t1`).Check(testkit.RowsWithSep("|", "def||10|3.14|2018-01-01 00:00:00|2011-11-11 11:11:11")) + require.EqualError(t, tk.ExecToErr("select default(x) from t1"), "[planner:1054]Unknown column 'x' in 'field list'") tk.MustQuery("select default(a0) from (select a as a0 from t1) as t0").Check(testkit.Rows("def")) - err = tk.ExecToErr("select default(a0) from (select a+1 as a0 from t1) as t0") - c.Assert(err.Error(), Equals, "[table:1364]Field 'a0' doesn't have a default value") + require.EqualError(t, tk.ExecToErr("select default(a0) from (select a+1 as a0 from t1) as t0"), "[table:1364]Field 'a0' doesn't have a default value") tk.MustExec("create table t2(a varchar(10), b varchar(10))") tk.MustExec("insert into t2 values ('1', '1')") - err = tk.ExecToErr("select default(a) from t1, t2") - c.Assert(err.Error(), Equals, "[expression:1052]Column 'a' in field list is ambiguous") + require.EqualError(t, tk.ExecToErr("select default(a) from t1, t2"), "[expression:1052]Column 'a' in field list is ambiguous") + tk.MustQuery("select default(t1.a) from t1, t2").Check(testkit.Rows("def")) tk.MustExec(`create table t3( @@ -146,7 +110,7 @@ func (s *testExpressionRewriterSuite) TestDefaultFunction(c *C) { default(b) as defb, default(c) as defc, default(d) as defd - from t3`).Check(testutil.RowsWithSep("|", "2011-11-11 11:11:11|2011-11-11 11:11:11|2011-11-11 11:11:11.000000|current_timestamp")) + from t3`).Check(testkit.RowsWithSep("|", "2011-11-11 11:11:11|2011-11-11 11:11:11|2011-11-11 11:11:11.000000|current_timestamp")) tk.MustExec(`create table t4(a int default 1, b varchar(5))`) tk.MustExec(`insert into t4 values (0, 'B'), (1, 'B'), (2, 'B')`) @@ -180,15 +144,10 @@ func (s *testExpressionRewriterSuite) TestDefaultFunction(c *C) { tk.MustQuery(`select a from t8 order by default(b) * a`).Check(testkit.Rows("1", "0")) } -func (s *testExpressionRewriterSuite) TestCompareSubquery(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCompareSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists s") @@ -283,32 +242,22 @@ func (s *testExpressionRewriterSuite) TestCompareSubquery(c *C) { tk.MustQuery("select count(1) from table_40_utf8_4 where ( select count(1) from t where table_40_utf8_4.col_bit64_key_signed!=table_40_utf8_4.col_tinyint_key_unsigned)").Check(testkit.Rows("1")) } -func (s *testExpressionRewriterSuite) TestCheckFullGroupBy(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCheckFullGroupBy(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustQuery("select t1.a, (select max(t2.b) from t t2) from t t1").Check(testkit.Rows()) - err = tk.ExecToErr("select t1.a, (select t2.a, max(t2.b) from t t2) from t t1") - c.Assert(terror.ErrorEqual(err, core.ErrMixOfGroupFuncAndFields), IsTrue, Commentf("err %v", err)) + err := tk.ExecToErr("select t1.a, (select t2.a, max(t2.b) from t t2) from t t1") + require.True(t, terror.ErrorEqual(err, plannercore.ErrMixOfGroupFuncAndFields)) } -func (s *testExpressionRewriterSuite) TestPatternLikeToExpression(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestPatternLikeToExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustQuery("select 0 like 'a string';").Check(testkit.Rows("0")) tk.MustQuery("select 0.0 like 'a string';").Check(testkit.Rows("0")) tk.MustQuery("select 0 like '0.00';").Check(testkit.Rows("0")) @@ -318,16 +267,10 @@ func (s *testExpressionRewriterSuite) TestPatternLikeToExpression(c *C) { tk.MustQuery("select 0.00 like '0.00';").Check(testkit.Rows("1")) } -func (s *testExpressionRewriterSuite) TestIssue20007(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestIssue20007(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1 (c_int int, c_str varchar(40), c_datetime datetime, primary key(c_int));") @@ -341,16 +284,10 @@ func (s *testExpressionRewriterSuite) TestIssue20007(c *C) { } } -func (s *testExpressionRewriterSuite) TestIssue9869(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestIssue9869(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(a int, b bigint unsigned);") @@ -359,33 +296,21 @@ func (s *testExpressionRewriterSuite) TestIssue9869(c *C) { testkit.Rows("4572794622775114594 4572794622775114594", "18196094287899841997 -250649785809709619", "11120436154190595086 -7326307919518956530")) } -func (s *testExpressionRewriterSuite) TestIssue17652(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestIssue17652(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(x bigint unsigned);") tk.MustExec("insert into t values( 9999999703771440633);") - tk.MustQuery("select ifnull(max(x), 0) from t").Check( - testkit.Rows("9999999703771440633")) + tk.MustQuery("select ifnull(max(x), 0) from t").Check(testkit.Rows("9999999703771440633")) } -func (s *testExpressionRewriterSuite) TestCompareMultiFieldsInSubquery(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestCompareMultiFieldsInSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1, t2, t3, t4;") tk.MustExec("CREATE TABLE t1(c1 int, c2 int);") @@ -408,58 +333,35 @@ func (s *testExpressionRewriterSuite) TestCompareMultiFieldsInSubquery(c *C) { tk.MustExec("INSERT INTO t4 VALUES (1, 2);") tk.MustQuery("SELECT * FROM t3 WHERE (SELECT c1 FROM t3 LIMIT 1) != ALL(SELECT c1 FROM t4);").Check(testkit.Rows()) tk.MustQuery("SELECT * FROM t3 WHERE (SELECT c1, c2 FROM t3 LIMIT 1) != ALL(SELECT c1, c2 FROM t4);").Check(testkit.Rows()) - } -func (s *testExpressionRewriterSuite) TestIssue22818(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestIssue22818(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a time);") tk.MustExec("insert into t values(\"23:22:22\");") - tk.MustQuery("select * from t where a between \"23:22:22\" and \"23:22:22\"").Check( - testkit.Rows("23:22:22")) + tk.MustQuery("select * from t where a between \"23:22:22\" and \"23:22:22\"").Check(testkit.Rows("23:22:22")) } -func (s *testExpressionRewriterSuite) TestIssue24705(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestIssue24705(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1,t2;") tk.MustExec("create table t1 (c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci);") tk.MustExec("create table t2 (c_int int, c_str varchar(40) character set utf8 collate utf8_unicode_ci);") - err = tk.ExecToErr("select * from t1 where c_str < any (select c_str from t2 where c_int between 6 and 9);") - c.Assert(err.Error(), Equals, "[expression:1267]Illegal mix of collations (utf8_general_ci,IMPLICIT) and (utf8_unicode_ci,IMPLICIT) for operation '<'") + err := tk.ExecToErr("select * from t1 where c_str < any (select c_str from t2 where c_int between 6 and 9);") + require.EqualError(t, err, "[expression:1267]Illegal mix of collations (utf8_general_ci,IMPLICIT) and (utf8_unicode_ci,IMPLICIT) for operation '<'") } -func (s *testExpressionRewriterSuiteSerial) TestBetweenExprCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestBetweenExprCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(a char(10) charset latin1 collate latin1_bin, c char(10) collate utf8mb4_general_ci);") @@ -467,20 +369,13 @@ func (s *testExpressionRewriterSuiteSerial) TestBetweenExprCollation(c *C) { tk.MustExec("insert into t1 values ('c', 'D');") tk.MustQuery("select * from t1 where a between 'B' and c;").Check(testkit.Rows("c D")) tk.MustQuery("explain select * from t1 where 'a' between 'g' and 'f';").Check(testkit.Rows("TableDual_6 0.00 root rows:0")) - tk.MustGetErrMsg("select * from t1 where a between 'B' collate utf8mb4_general_ci and c collate utf8mb4_unicode_ci;", "[expression:1270]Illegal mix of collations (latin1_bin,IMPLICIT), (utf8mb4_general_ci,EXPLICIT), (utf8mb4_unicode_ci,EXPLICIT) for operation 'BETWEEN'") } -func (s *testExpressionRewriterSuite) TestInsertOnDuplicateLazyMoreThan1Row(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestInsertOnDuplicateLazyMoreThan1Row(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("DROP TABLE if exists t1, t2, source;") tk.MustExec("CREATE TABLE t1(a INTEGER PRIMARY KEY);") @@ -496,15 +391,10 @@ func (s *testExpressionRewriterSuite) TestInsertOnDuplicateLazyMoreThan1Row(c *C tk.MustExec("DROP TABLE if exists t1, t2, source;") } -func (s *testExpressionRewriterSuite) TestMultiColInExpression(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestMultiColInExpression(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") @@ -519,12 +409,13 @@ func (s *testExpressionRewriterSuite) TestMultiColInExpression(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + expressionRewriterSuiteData := plannercore.GetExpressionRewriterSuiteData() + expressionRewriterSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) diff --git a/planner/core/integration_partition_test.go b/planner/core/integration_partition_test.go index d463d05c45754..e9a82d9f96169 100644 --- a/planner/core/integration_partition_test.go +++ b/planner/core/integration_partition_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" - "github.com/pingcap/tidb/util/israce" "github.com/stretchr/testify/require" ) @@ -182,10 +181,6 @@ func TestListPartitionFunctions(t *testing.T) { } func TestListPartitionOrderLimit(t *testing.T) { - if israce.RaceEnabled { - t.Skip("skip race test") - } - store, clean := testkit.CreateMockStore(t) defer clean() @@ -245,10 +240,6 @@ func TestListPartitionOrderLimit(t *testing.T) { } func TestListPartitionAgg(t *testing.T) { - if israce.RaceEnabled { - t.Skip("skip race test") - } - store, clean := testkit.CreateMockStore(t) defer clean() @@ -893,10 +884,6 @@ func TestListPartitionAlterPK(t *testing.T) { } func TestListPartitionRandomTransaction(t *testing.T) { - if israce.RaceEnabled { - t.Skip("skip race test") - } - store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 26a53d2ab012c..d13126973c1cc 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -18,14 +18,13 @@ import ( "bytes" "fmt" "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -36,72 +35,15 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/table" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testIntegrationSuite{}) -var _ = SerialSuites(&testIntegrationSerialSuite{}) - -type testIntegrationSuite struct { - testData testutil.TestData - store kv.Storage - dom *domain.Domain -} - -func (s *testIntegrationSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "integration_suite") - c.Assert(err, IsNil) -} - -func (s *testIntegrationSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testIntegrationSuite) SetUpTest(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testIntegrationSuite) TearDownTest(c *C) { - s.dom.Close() - err := s.store.Close() - c.Assert(err, IsNil) -} - -type testIntegrationSerialSuite struct { - testData testutil.TestData - store kv.Storage - dom *domain.Domain -} - -func (s *testIntegrationSerialSuite) SetUpSuite(c *C) { - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "integration_serial_suite") - c.Assert(err, IsNil) -} - -func (s *testIntegrationSerialSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testIntegrationSerialSuite) SetUpTest(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testIntegrationSerialSuite) TearDownTest(c *C) { - s.dom.Close() - err := s.store.Close() - c.Assert(err, IsNil) -} - -func (s *testIntegrationSuite) TestShowSubquery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestShowSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a varchar(10), b int, c int)") @@ -131,8 +73,10 @@ func (s *testIntegrationSuite) TestShowSubquery(c *C) { )) } -func (s *testIntegrationSuite) TestPpdWithSetVar(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPpdWithSetVar(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(c1 int, c2 varchar(255))") @@ -142,8 +86,10 @@ func (s *testIntegrationSuite) TestPpdWithSetVar(c *C) { tk.MustQuery("select t01.c1,t01.c2,t01.c3 from (select t1.*,@c3:=@c3+1 as c3 from (select t.*,@c3:=0 from t order by t.c1)t1)t01 where t01.c3=2 and t01.c2='d'").Check(testkit.Rows("2 d 2")) } -func (s *testIntegrationSuite) TestBitColErrorMessage(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBitColErrorMessage(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists bit_col_t") @@ -155,8 +101,10 @@ func (s *testIntegrationSuite) TestBitColErrorMessage(c *C) { tk.MustGetErrCode("create table bit_col_t (a bit(65))", mysql.ErrTooBigDisplaywidth) } -func (s *testIntegrationSuite) TestPushLimitDownIndexLookUpReader(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPushLimitDownIndexLookUpReader(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@session.tidb_executor_concurrency = 4;") tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") @@ -172,18 +120,21 @@ func (s *testIntegrationSuite) TestPushLimitDownIndexLookUpReader(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestAggColumnPrune(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAggColumnPrune(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -195,34 +146,54 @@ func (s *testIntegrationSuite) TestAggColumnPrune(c *C) { SQL string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSuite) TestIsFromUnixtimeNullRejective(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIsFromUnixtimeNullRejective(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a bigint, b bigint);`) - s.runTestsWithTestData("TestIsFromUnixtimeNullRejective", tk, c) + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } } -func (s *testIntegrationSuite) TestIssue22298(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22298(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t;`) tk.MustExec(`create table t(a int, b int);`) tk.MustGetErrMsg(`select * from t where 0 and c = 10;`, "[planner:1054]Unknown column 'c' in 'where clause'") } -func (s *testIntegrationSuite) TestIssue24571(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24571(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create view v as select 1 as b;`) tk.MustExec(`create table t (a int);`) @@ -235,8 +206,10 @@ func (s *testIntegrationSuite) TestIssue24571(c *C) { tk.MustExec("update (select 1 as a) as t, test.t set test.t.a=1;") } -func (s *testIntegrationSuite) TestBuildUpdateListResolver(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBuildUpdateListResolver(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // For issue https://github.com/pingcap/tidb/issues/24567 @@ -257,38 +230,20 @@ func (s *testIntegrationSuite) TestBuildUpdateListResolver(c *C) { tk.MustExec("drop table if exists t") } -func (s *testIntegrationSuite) TestIssue22828(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22828(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t1;`) tk.MustExec(`create table t (c int);`) tk.MustGetErrMsg(`select group_concat((select concat(c,group_concat(c)) FROM t where xxx=xxx)) FROM t;`, "[planner:1054]Unknown column 'xxx' in 'where clause'") } -func (s *testIntegrationSuite) runTestsWithTestData(caseName string, tk *testkit.TestKit, c *C) { - var input []string - var output []struct { - SQL string - Plan []string - } - s.testData.GetTestCasesByName(caseName, c, &input, &output) - for i, tt := range input { - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - } -} - -func (s *testIntegrationSuite) TestJoinNotNullFlag(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestJoinNotNullFlag(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(x int not null)") @@ -300,14 +255,10 @@ func (s *testIntegrationSuite) TestJoinNotNullFlag(c *C) { tk.MustQuery("select ifnull(t1.x, 'xxx') from t2 natural left join t1").Check(testkit.Rows("xxx")) } -func (s *testIntegrationSuite) TestAntiJoinConstProp(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestAntiJoinConstProp(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int not null)") @@ -345,8 +296,10 @@ func (s *testIntegrationSuite) TestAntiJoinConstProp(c *C) { )) } -func (s *testIntegrationSuite) TestSimplifyOuterJoinWithCast(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSimplifyOuterJoinWithCast(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -357,18 +310,21 @@ func (s *testIntegrationSuite) TestSimplifyOuterJoinWithCast(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestNoneAccessPathsFoundByIsolationRead(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNoneAccessPathsFoundByIsolationRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -384,8 +340,7 @@ func (s *testIntegrationSerialSuite) TestNoneAccessPathsFoundByIsolationRead(c * "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo")) _, err := tk.Exec("select * from t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tikv'. Please check tiflash replica or ensure the query is readonly.") + require.EqualError(t, err, "[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tikv'. Please check tiflash replica or ensure the query is readonly.") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash, tikv'") tk.MustExec("select * from t") @@ -397,17 +352,19 @@ func (s *testIntegrationSerialSuite) TestNoneAccessPathsFoundByIsolationRead(c * tk.MustExec("select * from t") } -func (s *testIntegrationSerialSuite) TestSelPushDownTiFlash(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelPushDownTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(20))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -425,19 +382,22 @@ func (s *testIntegrationSerialSuite) TestSelPushDownTiFlash(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestVerboseExplain(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestVerboseExplain(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) tk.MustExec("drop table if exists t1, t2, t3") @@ -458,10 +418,10 @@ func (s *testIntegrationSerialSuite) TestVerboseExplain(c *C) { tk.MustExec("analyze table t3") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -476,28 +436,31 @@ func (s *testIntegrationSerialSuite) TestVerboseExplain(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestPushDownToTiFlashWithKeepOrder(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPushDownToTiFlashWithKeepOrder(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(20))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -514,19 +477,22 @@ func (s *testIntegrationSerialSuite) TestPushDownToTiFlashWithKeepOrder(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists d1_t") tk.MustExec("create table d1_t(d1_k int, value int)") @@ -547,10 +513,10 @@ func (s *testIntegrationSerialSuite) TestMPPJoin(c *C) { tk.MustExec("analyze table fact_t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -567,19 +533,73 @@ func (s *testIntegrationSerialSuite) TestMPPJoin(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func TestMPPLeftSemiJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("create table test.t(a int not null, b int null);") + tk.MustExec("set tidb_allow_mpp=1; set tidb_enforce_mpp=1;") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } -func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForBroadcastJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPOuterJoinBuildSideForBroadcastJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists a") tk.MustExec("create table a(id int, value int)") @@ -590,10 +610,10 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForBroadcastJoin(c tk.MustExec("insert into b values(1,2),(2,3),(3,4)") tk.MustExec("analyze table b") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -611,19 +631,22 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForBroadcastJoin(c SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists a") tk.MustExec("create table a(id int, value int)") @@ -634,10 +657,10 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoinWith tk.MustExec("insert into b values(1,2),(2,3),(3,4)") tk.MustExec("analyze table b") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -655,19 +678,22 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoinWith SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPOuterJoinBuildSideForShuffleJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists a") tk.MustExec("create table a(id int, value int)") @@ -678,10 +704,10 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoin(c * tk.MustExec("insert into b values(1,2),(2,3),(3,4)") tk.MustExec("analyze table b") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "a" || tblInfo.Name.L == "b" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -699,19 +725,22 @@ func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoin(c * SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPShuffledJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists d1_t") tk.MustExec("create table d1_t(d1_k int, value int)") @@ -737,10 +766,10 @@ func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { tk.MustExec("analyze table fact_t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -759,19 +788,22 @@ func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(id int, v1 decimal(20,2), v2 decimal(20,2))") @@ -784,10 +816,10 @@ func (s *testIntegrationSerialSuite) TestMPPJoinWithCanNotFoundColumnInSchemaCol tk.MustExec("analyze table t2") tk.MustExec("analyze table t3") - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t1" || tblInfo.Name.L == "t2" || tblInfo.Name.L == "t3" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -808,19 +840,22 @@ func (s *testIntegrationSerialSuite) TestMPPJoinWithCanNotFoundColumnInSchemaCol SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinNotSupportedByTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists table_1") tk.MustExec("create table table_1(id int not null, bit_col bit(2) not null, datetime_col datetime not null)") @@ -828,10 +863,10 @@ func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { tk.MustExec("analyze table table_1") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "table_1" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -848,11 +883,12 @@ func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) @@ -860,11 +896,11 @@ func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 1") tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 1") - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) @@ -876,21 +912,21 @@ func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { tk.MustExec("set @@session.tidb_opt_broadcast_join = 1") // make cbo force choose broadcast join since sql hint does not work for semi/anti-semi join tk.MustExec("set @@session.tidb_opt_cpu_factor=10000000;") - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPWithHashExchangeUnderNewCollation(c *C) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - tk := testkit.NewTestKit(c, s.store) +func TestMPPWithHashExchangeUnderNewCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists table_1") tk.MustExec("create table table_1(id int not null, value char(10)) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci;") @@ -902,10 +938,10 @@ func (s *testIntegrationSerialSuite) TestMPPWithHashExchangeUnderNewCollation(c tk.MustExec("analyze table table_2") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "table_1" || tblInfo.Name.L == "table_2" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -926,20 +962,22 @@ func (s *testIntegrationSerialSuite) TestMPPWithHashExchangeUnderNewCollation(c SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollation(c *C) { - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) +func TestMPPWithBroadcastExchangeUnderNewCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists table_1") tk.MustExec("create table table_1(id int not null, value char(10))") @@ -947,10 +985,10 @@ func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollati tk.MustExec("analyze table table_1") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "table_1" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -960,7 +998,6 @@ func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollati } } - collate.SetNewCollationEnabledForTest(true) tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") var input []string @@ -968,21 +1005,22 @@ func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollati SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestPartitionTableDynamicModeUnderNewCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKitWithInit(c, s.store) +func TestPartitionTableDynamicModeUnderNewCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database test_new_collation") tk.MustExec("use test_new_collation") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -1019,9 +1057,10 @@ func (s *testIntegrationSerialSuite) TestPartitionTableDynamicModeUnderNewCollat tk.MustQuery(`select * from strlist where a in ('D', 'e')`).Sort().Check(testkit.Rows("D 1", "d 1", "e 1")) } -func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) +func TestMPPAvgRewrite(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists table_1") tk.MustExec("create table table_1(id int not null, value decimal(10,2))") @@ -1029,10 +1068,10 @@ func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { tk.MustExec("analyze table table_1") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "table_1" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -1042,7 +1081,6 @@ func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { } } - collate.SetNewCollationEnabledForTest(true) tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@session.tidb_allow_mpp = 1") var input []string @@ -1050,28 +1088,31 @@ func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestAggPushDownEngine(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAggPushDownEngine(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(20))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -1083,22 +1124,24 @@ func (s *testIntegrationSerialSuite) TestAggPushDownEngine(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustQuery("desc select approx_count_distinct(a) from t").Check(testkit.Rows( - "HashAgg_11 1.00 root funcs:approx_count_distinct(Column#4)->Column#3", - "└─TableReader_12 1.00 root data:HashAgg_6", - " └─HashAgg_6 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#4", - " └─TableFullScan_10 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:approx_count_distinct(Column#5)->Column#3", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:approx_count_distinct(test.t.a)->Column#5", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") - tk.MustQuery("desc select approx_count_distinct(a) from t").Check(testkit.Rows( - "HashAgg_5 1.00 root funcs:approx_count_distinct(test.t.a)->Column#3", - "└─TableReader_11 10000.00 root data:TableFullScan_10", - " └─TableFullScan_10 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) + tk.MustQuery("explain format = 'brief' select approx_count_distinct(a) from t").Check(testkit.Rows( + "HashAgg 1.00 root funcs:approx_count_distinct(test.t.a)->Column#3", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) } -func (s *testIntegrationSerialSuite) TestIssue15110(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15110(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists crm_rd_150m") tk.MustExec(`CREATE TABLE crm_rd_150m ( @@ -1115,10 +1158,10 @@ func (s *testIntegrationSerialSuite) TestIssue15110(c *C) { ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;`) // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "crm_rd_150m" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -1132,8 +1175,10 @@ func (s *testIntegrationSerialSuite) TestIssue15110(c *C) { tk.MustExec("explain format = 'brief' SELECT count(*) FROM crm_rd_150m dataset_48 WHERE (CASE WHEN (month(dataset_48.customer_first_date)) <= 30 THEN '新客' ELSE NULL END) IS NOT NULL;") } -func (s *testIntegrationSerialSuite) TestReadFromStorageHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReadFromStorageHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, tt, ttt") @@ -1143,10 +1188,10 @@ func (s *testIntegrationSerialSuite) TestReadFromStorageHint(c *C) { tk.MustExec("create table ttt(a int, primary key (a desc))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1160,21 +1205,24 @@ func (s *testIntegrationSerialSuite) TestReadFromStorageHint(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } -func (s *testIntegrationSerialSuite) TestReadFromStorageHintAndIsolationRead(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReadFromStorageHintAndIsolationRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, tt, ttt") @@ -1182,10 +1230,10 @@ func (s *testIntegrationSerialSuite) TestReadFromStorageHintAndIsolationRead(c * tk.MustExec("set @@session.tidb_isolation_read_engines=\"tikv\"") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1199,32 +1247,35 @@ func (s *testIntegrationSerialSuite) TestReadFromStorageHintAndIsolationRead(c * Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - tk.Se.GetSessionVars().StmtCtx.SetWarnings(nil) - s.testData.OnRecord(func() { + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } -func (s *testIntegrationSerialSuite) TestIsolationReadTiFlashNotChoosePointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIsolationReadTiFlashNotChoosePointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, primary key (a))") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1238,28 +1289,31 @@ func (s *testIntegrationSerialSuite) TestIsolationReadTiFlashNotChoosePointGet(c SQL string Result []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } -func (s *testIntegrationSerialSuite) TestIsolationReadTiFlashUseIndexHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIsolationReadTiFlashUseIndexHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, index idx(a));") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ Count: 1, @@ -1274,21 +1328,24 @@ func (s *testIntegrationSerialSuite) TestIsolationReadTiFlashUseIndexHint(c *C) Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } -func (s *testIntegrationSerialSuite) TestIsolationReadDoNotFilterSystemDB(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIsolationReadDoNotFilterSystemDB(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_isolation_read_engines = \"tiflash\"") @@ -1297,19 +1354,22 @@ func (s *testIntegrationSerialSuite) TestIsolationReadDoNotFilterSystemDB(c *C) SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestPartitionTableStats(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionTableStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) { tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) tk.MustExec("use test") @@ -1324,19 +1384,22 @@ func (s *testIntegrationSuite) TestPartitionTableStats(c *C) { SQL string Result []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } } -func (s *testIntegrationSuite) TestPartitionPruningForInExpr(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionPruningForInExpr(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1348,18 +1411,21 @@ func (s *testIntegrationSuite) TestPartitionPruningForInExpr(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestPartitionPruningWithDateType(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionPruningWithDateType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1369,54 +1435,60 @@ func (s *testIntegrationSerialSuite) TestPartitionPruningWithDateType(c *C) { // cannot get the statistical information immediately // tk.MustQuery(`SELECT PARTITION_NAME,TABLE_ROWS FROM INFORMATION_SCHEMA.PARTITIONS WHERE TABLE_NAME = 't';`).Check(testkit.Rows("p1 1", "p2 1")) str := tk.MustQuery(`desc select * from t where a < '2000-01-01';`).Rows()[0][3].(string) - c.Assert(strings.Contains(str, "partition:p1"), IsTrue) + require.True(t, strings.Contains(str, "partition:p1")) } -func (s *testIntegrationSuite) TestPartitionPruningForEQ(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionPruningForEQ(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") - is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) pt := tbl.(table.PartitionedTable) - query, err := expression.ParseSimpleExprWithTableInfo(tk.Se, "a = '2020-01-01 00:00:00'", tbl.Meta()) - c.Assert(err, IsNil) - dbName := model.NewCIStr(tk.Se.GetSessionVars().CurrentDB) - columns, names, err := expression.ColumnInfos2ColumnsAndNames(tk.Se, dbName, tbl.Meta().Name, tbl.Meta().Cols(), tbl.Meta()) - c.Assert(err, IsNil) + query, err := expression.ParseSimpleExprWithTableInfo(tk.Session(), "a = '2020-01-01 00:00:00'", tbl.Meta()) + require.NoError(t, err) + dbName := model.NewCIStr(tk.Session().GetSessionVars().CurrentDB) + columns, names, err := expression.ColumnInfos2ColumnsAndNames(tk.Session(), dbName, tbl.Meta().Name, tbl.Meta().Cols(), tbl.Meta()) + require.NoError(t, err) // Even the partition is not monotonous, EQ condition should be prune! // select * from t where a = '2020-01-01 00:00:00' - res, err := core.PartitionPruning(tk.Se, pt, []expression.Expression{query}, nil, columns, names) - c.Assert(err, IsNil) - c.Assert(res, HasLen, 1) - c.Assert(res[0], Equals, 0) + res, err := core.PartitionPruning(tk.Session(), pt, []expression.Expression{query}, nil, columns, names) + require.NoError(t, err) + require.Len(t, res, 1) + require.Equal(t, 0, res[0]) } -func (s *testIntegrationSuite) TestErrNoDB(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestErrNoDB(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create user test") _, err := tk.Exec("grant select on test1111 to test@'%'") - c.Assert(errors.Cause(err), Equals, core.ErrNoDB) + require.Equal(t, core.ErrNoDB, errors.Cause(err)) _, err = tk.Exec("grant select on * to test@'%'") - c.Assert(errors.Cause(err), Equals, core.ErrNoDB) + require.Equal(t, core.ErrNoDB, errors.Cause(err)) _, err = tk.Exec("revoke select on * from test@'%'") - c.Assert(errors.Cause(err), Equals, core.ErrNoDB) + require.Equal(t, core.ErrNoDB, errors.Cause(err)) tk.MustExec("use test") tk.MustExec("create table test1111 (id int)") tk.MustExec("grant select on test1111 to test@'%'") } -func (s *testIntegrationSuite) TestMaxMinEliminate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMaxMinEliminate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key)") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table cluster_index_t(a int, b int, c int, primary key (a, b));") var input []string @@ -1424,18 +1496,21 @@ func (s *testIntegrationSuite) TestMaxMinEliminate(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestINLJHintSmallTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestINLJHintSmallTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int, key(a))") @@ -1446,12 +1521,14 @@ func (s *testIntegrationSuite) TestINLJHintSmallTable(c *C) { tk.MustExec("explain format = 'brief' select /*+ TIDB_INLJ(t1) */ * from t1 join t2 on t1.a = t2.a") } -func (s *testIntegrationSuite) TestIndexJoinUniqueCompositeIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexJoinUniqueCompositeIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table t1(a int not null, c int not null)") tk.MustExec("create table t2(a int not null, b int not null, c int not null, primary key(a,b))") tk.MustExec("insert into t1 values(1,1)") @@ -1463,18 +1540,21 @@ func (s *testIntegrationSuite) TestIndexJoinUniqueCompositeIndex(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIndexMerge(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMerge(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1485,18 +1565,21 @@ func (s *testIntegrationSuite) TestIndexMerge(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIndexMergeHint4CNF(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeHint4CNF(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1507,18 +1590,21 @@ func (s *testIntegrationSuite) TestIndexMergeHint4CNF(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestInvisibleIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInvisibleIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1532,9 +1618,9 @@ func (s *testIntegrationSuite) TestInvisibleIndex(c *C) { // Optimizer cannot use invisible indexes. tk.MustQuery("select a from t order by a").Check(testkit.Rows("1")) - c.Check(tk.MustUseIndex("select a from t order by a", "i_a"), IsFalse) + require.False(t, tk.MustUseIndex("select a from t order by a", "i_a")) tk.MustQuery("select a from t where a > 0").Check(testkit.Rows("1")) - c.Check(tk.MustUseIndex("select a from t where a > 1", "i_a"), IsFalse) + require.False(t, tk.MustUseIndex("select a from t where a > 1", "i_a")) // If use invisible indexes in index hint and sql hint, throw an error. errStr := "[planner:1176]Key 'i_a' doesn't exist in table 't'" @@ -1542,27 +1628,29 @@ func (s *testIntegrationSuite) TestInvisibleIndex(c *C) { tk.MustGetErrMsg("select * from t force index(i_a)", errStr) tk.MustGetErrMsg("select * from t ignore index(i_a)", errStr) tk.MustQuery("select /*+ USE_INDEX(t, i_a) */ * from t") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, errStr) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, errStr) tk.MustQuery("select /*+ IGNORE_INDEX(t, i_a), USE_INDEX(t, i_b) */ a from t order by a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, errStr) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, errStr) tk.MustQuery("select /*+ FORCE_INDEX(t, i_a), USE_INDEX(t, i_b) */ a from t order by a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, errStr) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, errStr) // For issue 15519 inapplicableErrStr := "[planner:1815]force_index(test.aaa) is inapplicable, check whether the table(test.aaa) exists" tk.MustQuery("select /*+ FORCE_INDEX(aaa) */ * from t") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, inapplicableErrStr) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, inapplicableErrStr) tk.MustExec("admin check table t") tk.MustExec("admin check index t i_a") } // for issue #14822 -func (s *testIntegrationSuite) TestIndexJoinTableRange(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexJoinTableRange(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -1574,26 +1662,31 @@ func (s *testIntegrationSuite) TestIndexJoinTableRange(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestTopNByConstFunc(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTopNByConstFunc(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustQuery("select max(t.col) from (select 'a' as col union all select '' as col) as t").Check(testkit.Rows( "a", )) } -func (s *testIntegrationSuite) TestSubqueryWithTopN(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSubqueryWithTopN(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1604,18 +1697,21 @@ func (s *testIntegrationSuite) TestSubqueryWithTopN(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIndexHintWarning(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexHintWarning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, key a(a), key b(b))") @@ -1625,29 +1721,32 @@ func (s *testIntegrationSuite) TestIndexHintWarning(c *C) { SQL string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt tk.MustQuery(tt) - warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() output[i].Warnings = make([]string, len(warns)) for j := range warns { output[i].Warnings[j] = warns[j].Err.Error() } }) tk.MustQuery(tt) - warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(len(warns), Equals, len(output[i].Warnings)) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warns, len(output[i].Warnings)) for j := range warns { - c.Assert(warns[j].Level, Equals, stmtctx.WarnLevelWarning) - c.Assert(warns[j].Err.Error(), Equals, output[i].Warnings[j]) + require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) + require.EqualError(t, warns[j].Err, output[i].Warnings[j]) } } } -func (s *testIntegrationSuite) TestIssue15546(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15546(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, pt, vt") @@ -1660,8 +1759,10 @@ func (s *testIntegrationSuite) TestIssue15546(c *C) { tk.MustQuery("select * from pt, vt where pt.a = vt.a").Check(testkit.Rows("1 1 1 1")) } -func (s *testIntegrationSuite) TestApproxCountDistinctInPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestApproxCountDistinctInPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1681,8 +1782,10 @@ func (s *testIntegrationSuite) TestApproxCountDistinctInPartitionTable(c *C) { tk.MustQuery("select approx_count_distinct(a), b from t group by b order by b desc").Check(testkit.Rows("1 2", "3 1")) } -func (s *testIntegrationSuite) TestApproxPercentile(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestApproxPercentile(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1695,20 +1798,23 @@ func (s *testIntegrationSuite) TestApproxPercentile(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSuite) TestIssue17813(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue17813(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists hash_partition_overflow") @@ -1718,8 +1824,10 @@ func (s *testIntegrationSuite) TestIssue17813(c *C) { tk.MustQuery("select * from hash_partition_overflow where c0 in (1, 9223372036854775808)").Check(testkit.Rows("9223372036854775808")) } -func (s *testIntegrationSuite) TestHintWithRequiredProperty(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHintWithRequiredProperty(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@session.tidb_executor_concurrency = 4;") tk.MustExec("set @@session.tidb_hash_join_concurrency = 5;") tk.MustExec("set @@session.tidb_distsql_scan_concurrency = 15;") @@ -1732,28 +1840,31 @@ func (s *testIntegrationSuite) TestHintWithRequiredProperty(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() output[i].Warnings = make([]string, len(warnings)) for j, warning := range warnings { output[i].Warnings[j] = warning.Err.Error() } }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(len(warnings), Equals, len(output[i].Warnings)) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warnings, len(output[i].Warnings)) for j, warning := range warnings { - c.Assert(output[i].Warnings[j], Equals, warning.Err.Error()) + require.EqualError(t, warning.Err, output[i].Warnings[j]) } } } -func (s *testIntegrationSuite) TestIssue15813(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15813(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0, t1") @@ -1764,8 +1875,10 @@ func (s *testIntegrationSuite) TestIssue15813(c *C) { tk.MustQuery("select /*+ MERGE_JOIN(t0, t1) */ * from t0, t1 where t0.c0 = t1.c0").Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestIssue31261(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue31261(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists PK_MULTI_COL_5177`) @@ -1779,23 +1892,27 @@ func (s *testIntegrationSuite) TestIssue31261(c *C) { KEY IDX3 (COL3), KEY IDXM (COL3,COL2))`) tk.MustExec(`insert into PK_MULTI_COL_5177(col1, col2, col3) values(0x00000000000000000000, 0x002B200DF5BA03E59F82, 1)`) - c.Assert(len(tk.MustQuery(`select col1, col2 from PK_MULTI_COL_5177 where col1 = 0x00000000000000000000 and col2 in (0x002B200DF5BA03E59F82, 0x002B200DF5BA03E59F82, 0x002B200DF5BA03E59F82)`).Rows()), Equals, 1) - c.Assert(len(tk.MustQuery(`select col1, col2 from PK_MULTI_COL_5177 where col1 = 0x00000000000000000000 and col2 = 0x002B200DF5BA03E59F82`).Rows()), Equals, 1) + require.Len(t, tk.MustQuery(`select col1, col2 from PK_MULTI_COL_5177 where col1 = 0x00000000000000000000 and col2 in (0x002B200DF5BA03E59F82, 0x002B200DF5BA03E59F82, 0x002B200DF5BA03E59F82)`).Rows(), 1) + require.Len(t, tk.MustQuery(`select col1, col2 from PK_MULTI_COL_5177 where col1 = 0x00000000000000000000 and col2 = 0x002B200DF5BA03E59F82`).Rows(), 1) } -func (s *testIntegrationSuite) TestFullGroupByOrderBy(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestFullGroupByOrderBy(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustQuery("select count(a) as b from t group by a order by b").Check(testkit.Rows()) err := tk.ExecToErr("select count(a) as cnt from t group by a order by b") - c.Assert(terror.ErrorEqual(err, core.ErrFieldNotInGroupBy), IsTrue) + require.True(t, terror.ErrorEqual(err, core.ErrFieldNotInGroupBy)) } -func (s *testIntegrationSuite) TestHintWithoutTableWarning(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHintWithoutTableWarning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, key a(a))") @@ -1805,37 +1922,42 @@ func (s *testIntegrationSuite) TestHintWithoutTableWarning(c *C) { SQL string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt tk.MustQuery(tt) - warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings() + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() output[i].Warnings = make([]string, len(warns)) for j := range warns { output[i].Warnings[j] = warns[j].Err.Error() } }) tk.MustQuery(tt) - warns := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(len(warns), Equals, len(output[i].Warnings)) + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warns, len(output[i].Warnings)) for j := range warns { - c.Assert(warns[j].Level, Equals, stmtctx.WarnLevelWarning) - c.Assert(warns[j].Err.Error(), Equals, output[i].Warnings[j]) + require.Equal(t, stmtctx.WarnLevelWarning, warns[j].Level) + require.EqualError(t, warns[j].Err, output[i].Warnings[j]) } } } -func (s *testIntegrationSuite) TestIssue15858(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15858(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key)") tk.MustExec("select * from t t1, (select a from t order by a+1) t2 where t1.a = t2.a") } -func (s *testIntegrationSuite) TestIssue15846(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue15846(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0, t1") tk.MustExec("CREATE TABLE t0(t0 INT UNIQUE);") @@ -1860,8 +1982,10 @@ func (s *testIntegrationSuite) TestIssue15846(c *C) { tk.MustQuery("SELECT t1.c0 FROM t1 LEFT JOIN t0 ON 1;").Check(testkit.Rows("0", "0")) } -func (s *testIntegrationSuite) TestFloorUnixTimestampPruning(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestFloorUnixTimestampPruning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists floor_unix_timestamp") tk.MustExec(`create table floor_unix_timestamp (ts timestamp(3)) @@ -1881,8 +2005,10 @@ partition p2 values less than (unix_timestamp('2020-04-15 00:00:00')))`) tk.MustQuery("select * from floor_unix_timestamp partition(p1, p2) where ts > '2020-04-14 00:00:00'").Check(testkit.Rows("2020-04-14 00:00:42.000")) } -func (s *testIntegrationSuite) TestIssue16290And16292(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16290And16292(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b int, primary key(a));") @@ -1898,8 +2024,10 @@ func (s *testIntegrationSuite) TestIssue16290And16292(c *C) { } } -func (s *testIntegrationSuite) TestTableDualWithRequiredProperty(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTableDualWithRequiredProperty(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("create table t1 (a int, b int) partition by range(a) " + @@ -1908,8 +2036,10 @@ func (s *testIntegrationSuite) TestTableDualWithRequiredProperty(c *C) { tk.MustExec("select /*+ MERGE_JOIN(t1, t2) */ * from t1 partition (p0), t2 where t1.a > 100 and t1.a = t2.a") } -func (s *testIntegrationSuite) TestIndexJoinInnerIndexNDV(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexJoinInnerIndexNDV(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int not null, c int not null)") @@ -1923,18 +2053,21 @@ func (s *testIntegrationSuite) TestIndexJoinInnerIndexNDV(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestIssue16837(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16837(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b int,c int,d int,e int,unique key idx_ab(a,b),unique key(c),unique key(d))") @@ -1949,8 +2082,10 @@ func (s *testIntegrationSerialSuite) TestIssue16837(c *C) { tk.MustQuery("select /*+ use_index_merge(t,c,idx_ab) */ * from t where a = 1 or (e = 1 and c = 1)").Check(testkit.Rows()) } -func (s *testIntegrationSerialSuite) TestIndexMerge(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeSerial(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, unique key(a), unique key(b))") @@ -1964,20 +2099,23 @@ func (s *testIntegrationSerialSuite) TestIndexMerge(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) } } -func (s *testIntegrationSerialSuite) TestIndexMergePartialScansClusteredIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergePartialScansClusteredIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") @@ -2028,8 +2166,10 @@ func (s *testIntegrationSerialSuite) TestIndexMergePartialScansClusteredIndex(c } } -func (s *testIntegrationSerialSuite) TestIndexMergePartialScansTiDBRowID(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergePartialScansTiDBRowID(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") @@ -2080,8 +2220,10 @@ func (s *testIntegrationSerialSuite) TestIndexMergePartialScansTiDBRowID(c *C) { } } -func (s *testIntegrationSerialSuite) TestIndexMergePartialScansPKIsHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergePartialScansPKIsHandle(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") @@ -2128,8 +2270,10 @@ func (s *testIntegrationSerialSuite) TestIndexMergePartialScansPKIsHandle(c *C) } } -func (s *testIntegrationSerialSuite) TestIssue23919(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23919(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") // Test for the minimal reproducible case. @@ -2167,8 +2311,10 @@ PARTITION BY RANGE ( col_8 ) ( "t.col_8 = 7372 order by col_5,col_8 ) ordered_tbl group by col_6;").Check(testkit.Rows("")) } -func (s *testIntegrationSerialSuite) TestIssue16407(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16407(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int,b char(100),key(a),key(b(10)))") @@ -2183,8 +2329,10 @@ func (s *testIntegrationSerialSuite) TestIssue16407(c *C) { tk.MustQuery("select /*+ use_index_merge(t) */ * from t where a=10 or b='x'").Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestStreamAggProp(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestStreamAggProp(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2197,20 +2345,23 @@ func (s *testIntegrationSuite) TestStreamAggProp(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSuite) TestOptimizeHintOnPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOptimizeHintOnPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2228,10 +2379,10 @@ func (s *testIntegrationSuite) TestOptimizeHintOnPartitionTable(c *C) { }() // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -2249,30 +2400,33 @@ func (s *testIntegrationSuite) TestOptimizeHintOnPartitionTable(c *C) { Plan []string Warn []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Warn = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Warn = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warn...)) } } -func (s *testIntegrationSerialSuite) TestNotReadOnlySQLOnTiFlash(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNotReadOnlySQLOnTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b varchar(20))") tk.MustExec(`set @@tidb_isolation_read_engines = "tiflash"`) // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -2282,22 +2436,21 @@ func (s *testIntegrationSerialSuite) TestNotReadOnlySQLOnTiFlash(c *C) { } } err := tk.ExecToErr("select * from t for update") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) err = tk.ExecToErr("insert into t select * from t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) tk.MustExec("prepare stmt_insert from 'insert into t select * from t where t.a = ?'") tk.MustExec("set @a=1") err = tk.ExecToErr("execute stmt_insert using @a") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) + require.EqualError(t, err, `[planner:1815]Internal : No access path for table 't' is found with 'tidb_isolation_read_engines' = 'tiflash', valid values can be 'tiflash, tikv'. Please check tiflash replica or ensure the query is readonly.`) } -func (s *testIntegrationSuite) TestSelectLimit(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectLimit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2307,7 +2460,7 @@ func (s *testIntegrationSuite) TestSelectLimit(c *C) { // normal test tk.MustExec("set @@session.sql_select_limit=1") result := tk.MustQuery("select * from t order by a") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 0) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) result.Check(testkit.Rows("1")) result = tk.MustQuery("select * from t order by a limit 2") result.Check(testkit.Rows("1", "1")) @@ -2372,31 +2525,36 @@ func (s *testIntegrationSuite) TestSelectLimit(c *C) { result.Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestHintParserWarnings(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHintParserWarnings(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b int, key(a), key(b));") tk.MustExec("select /*+ use_index_merge() */ * from t where a = 1 or b = 1;") rows := tk.MustQuery("show warnings;").Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) } -func (s *testIntegrationSuite) TestIssue16935(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue16935(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0;") tk.MustExec("CREATE TABLE t0(c0 INT);") tk.MustExec("INSERT INTO t0(c0) VALUES (1), (1), (1), (1), (1), (1);") tk.MustExec("CREATE definer='root'@'localhost' VIEW v0(c0) AS SELECT NULL FROM t0;") - tk.MustQuery("SELECT * FROM t0 LEFT JOIN v0 ON TRUE WHERE v0.c0 IS NULL;") } -func (s *testIntegrationSuite) TestAccessPathOnClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestAccessPathOnClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") tk.MustExec(`insert into t1 values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) @@ -2408,24 +2566,27 @@ func (s *testIntegrationSuite) TestAccessPathOnClusterIndex(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) tk.MustQuery("explain format='brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSuite) TestClusterIndexUniqueDoubleRead(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusterIndexUniqueDoubleRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database cluster_idx_unique_double_read;") tk.MustExec("use cluster_idx_unique_double_read;") defer tk.MustExec("drop database cluster_idx_unique_double_read;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(64), b varchar(64), uk int, v int, primary key(a, b), unique key uuk(uk));") @@ -2433,10 +2594,12 @@ func (s *testIntegrationSuite) TestClusterIndexUniqueDoubleRead(c *C) { tk.MustQuery("select * from t use index (uuk);").Check(testkit.Rows("a a1 1 11", "b b1 2 22", "c c1 3 33")) } -func (s *testIntegrationSuite) TestIndexJoinOnClusteredIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexJoinOnClusteredIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1") tk.MustExec("create table t (a int, b varchar(20), c decimal(40,10), d int, primary key(a,b), key(c))") tk.MustExec(`insert into t values (1,"111",1.1,11), (2,"222",2.2,12), (3,"333",3.3,13)`) @@ -2448,22 +2611,25 @@ func (s *testIntegrationSuite) TestIndexJoinOnClusteredIndex(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery("explain format = 'brief'" + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSerialSuite) TestIssue18984(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue18984(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t, t2") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("create table t(a int, b int, c int, primary key(a, b))") tk.MustExec("create table t2(a int, b int, c int, d int, primary key(a,b), index idx(c))") tk.MustExec("insert into t values(1,1,1), (2,2,2), (3,3,3)") @@ -2479,8 +2645,223 @@ func (s *testIntegrationSerialSuite) TestIssue18984(c *C) { "3 3 3 2 4 3 5")) } -func (s *testIntegrationSuite) TestDistinctScalarFunctionPushDown(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestScalarFunctionPushDown(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(id int signed, id2 int unsigned ,c varchar(11), d datetime, b double)") + tk.MustExec("insert into t(id,c,d) values (1,'abc','2021-12-12')") + rows := [][]interface{}{ + {"TableReader_7", "root", "data:Selection_6"}, + {"└─Selection_6", "cop[tikv]", "right(test.t.c, 1)"}, + {" └─TableFullScan_5", "cop[tikv]", "keep order:false, stats:pseudo"}, + } + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where right(c,1);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "left(test.t.c, 1)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where left(c,1);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "mod(test.t.id, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id, id);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id, test.t.id2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id, id2);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id2, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id2, id);"). + CheckAt([]int{0, 3, 6}, rows) + rows[1][2] = "mod(test.t.id2, test.t.id2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where mod(id2, id2);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "sin(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where sin(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "asin(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where asin(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "cos(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where cos(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "acos(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where acos(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "tan(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where tan(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "atan(cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where atan(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "atan2(cast(test.t.id, double BINARY), cast(test.t.id, double BINARY))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where atan2(id,id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "hour(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where hour(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "hour(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where hour(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "minute(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where minute(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "second(cast(test.t.d, time))" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where second(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "month(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where month(d);"). + CheckAt([]int{0, 3, 6}, rows) + + //rows[1][2] = "dayname(test.t.d)" + //tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where dayname(d);"). + // CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "dayofmonth(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where dayofmonth(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "weekday(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where weekday(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "weekday(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where weekday(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "from_days(test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where from_days(id);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "to_days(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where to_days(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "last_day(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where last_day(d);"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "gt(4, test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where pi() > id;"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "truncate(test.t.id, 0)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where truncate(id,0)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "bin(test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where bin(id)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "unhex(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where unhex(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "locate(test.t.c, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where locate(c,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "ord(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where ord(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "lpad(test.t.c, 1, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where lpad(c,1,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "rpad(test.t.c, 1, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where rpad(c,1,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "trim(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where trim(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "from_base64(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where from_base64(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "to_base64(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where to_base64(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "make_set(1, test.t.c, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where make_set(1,c,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "substring_index(test.t.c, test.t.c, 1)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where substring_index(c,c,1)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "instr(test.t.c, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where instr(c,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "quote(test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where quote(c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "oct(test.t.id)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where oct(id)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "find_in_set(test.t.c, test.t.c)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where find_in_set(c,c)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "repeat(test.t.c, 2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where repeat(c,2)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "round(test.t.b)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where round(b)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "round(test.t.b, 2)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where round(b,2)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "date(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where date(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "week(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where week(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "yearweek(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where yearweek(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "to_seconds(test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where to_seconds(d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "datediff(test.t.d, test.t.d)" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where datediff(d,d)"). + CheckAt([]int{0, 3, 6}, rows) + + rows[1][2] = "gt(test.t.d, sysdate())" + tk.MustQuery("explain analyze select /*+read_from_storage(tikv[t])*/ * from t where d > sysdate()"). + CheckAt([]int{0, 3, 6}, rows) +} + +func TestDistinctScalarFunctionPushDown(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int not null, b int not null, c int not null, primary key (a,c)) partition by range (c) (partition p0 values less than (5), partition p1 values less than (10))") @@ -2490,8 +2871,10 @@ func (s *testIntegrationSuite) TestDistinctScalarFunctionPushDown(c *C) { )) } -func (s *testIntegrationSerialSuite) TestExplainAnalyzePointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplainAnalyzePointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(20))") @@ -2501,19 +2884,21 @@ func (s *testIntegrationSerialSuite) TestExplainAnalyzePointGet(c *C) { checkExplain := func(rpc string) { resBuff := bytes.NewBufferString("") for _, row := range res.Rows() { - fmt.Fprintf(resBuff, "%s\n", row) + _, _ = fmt.Fprintf(resBuff, "%s\n", row) } explain := resBuff.String() - c.Assert(strings.Contains(explain, rpc+":{num_rpc:"), IsTrue, Commentf("%s", explain)) - c.Assert(strings.Contains(explain, "total_time:"), IsTrue, Commentf("%s", explain)) + require.Containsf(t, explain, rpc+":{num_rpc:", "%s", explain) + require.Containsf(t, explain, "total_time:", "%s", explain) } checkExplain("Get") res = tk.MustQuery("explain analyze select * from t where a in (1,2,3);") checkExplain("BatchGet") } -func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplainAnalyzeDML(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(" create table t (a int, b int, unique index (a));") @@ -2523,19 +2908,21 @@ func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML(c *C) { checkExplain := func(rpc string) { resBuff := bytes.NewBufferString("") for _, row := range res.Rows() { - fmt.Fprintf(resBuff, "%s\n", row) + _, _ = fmt.Fprintf(resBuff, "%s\n", row) } explain := resBuff.String() - c.Assert(strings.Contains(explain, rpc+":{num_rpc:"), IsTrue, Commentf("%s", explain)) - c.Assert(strings.Contains(explain, "total_time:"), IsTrue, Commentf("%s", explain)) + require.Containsf(t, explain, rpc+":{num_rpc:", "%s", explain) + require.Containsf(t, explain, "total_time:", "%s", explain) } checkExplain("Get") res = tk.MustQuery("explain analyze insert ignore into t values (1,1),(2,2),(3,3),(4,4);") checkExplain("BatchGet") } -func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplainAnalyzeDML2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") cases := []struct { @@ -2609,10 +2996,10 @@ func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML2(c *C) { res := tk.MustQuery("explain analyze " + ca.sql) resBuff := bytes.NewBufferString("") for _, row := range res.Rows() { - fmt.Fprintf(resBuff, "%s\t", row) + _, _ = fmt.Fprintf(resBuff, "%s\t", row) } explain := resBuff.String() - c.Assert(explain, Matches, ca.planRegexp, Commentf("idx: %v,sql: %v", i, ca.sql)) + require.Regexpf(t, ca.planRegexp, explain, "idx: %v,sql: %v", i, ca.sql) } } @@ -2627,15 +3014,17 @@ func (s *testIntegrationSerialSuite) TestExplainAnalyzeDML2(c *C) { res := tk.MustQuery("explain analyze " + ca.sql) resBuff := bytes.NewBufferString("") for _, row := range res.Rows() { - fmt.Fprintf(resBuff, "%s\t", row) + _, _ = fmt.Fprintf(resBuff, "%s\t", row) } explain := resBuff.String() - c.Assert(strings.Contains(explain, "auto_id_allocator"), IsFalse, Commentf("sql: %v, explain: %v", ca.sql, explain)) + require.NotContainsf(t, explain, "auto_id_allocator", "sql: %v, explain: %v", ca.sql, explain) } } -func (s *testIntegrationSuite) TestPartitionExplain(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionExplain(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table pt (id int, c int, key i_id(id), key i_c(c)) partition by range (c) ( partition p0 values less than (4), @@ -2649,18 +3038,21 @@ partition p2 values less than (10))`) SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) }) tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestPartialBatchPointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartialBatchPointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c_int int, c_str varchar(40), primary key(c_int, c_str))") @@ -2673,8 +3065,10 @@ func (s *testIntegrationSuite) TestPartialBatchPointGet(c *C) { )) } -func (s *testIntegrationSuite) TestIssue19926(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue19926(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists ta;") tk.MustExec("drop table if exists tb;") @@ -2690,8 +3084,10 @@ func (s *testIntegrationSuite) TestIssue19926(c *C) { tk.MustQuery("SELECT tc.status,v.id FROM tc, v WHERE tc.id = v.id AND v.status = '11';").Check(testkit.Rows("1 1")) } -func (s *testIntegrationSuite) TestDeleteUsingJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDeleteUsingJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int primary key, b int)") @@ -2703,13 +3099,12 @@ func (s *testIntegrationSuite) TestDeleteUsingJoin(c *C) { tk.MustQuery("select * from t2").Check(testkit.Rows("2 2")) } -func (s *testIntegrationSerialSuite) Test19942(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - - tk := testkit.NewTestKit(c, s.store) +func Test19942(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("CREATE TABLE test.`t` (" + " `a` int(11) NOT NULL," + " `b` varchar(10) COLLATE utf8_general_ci NOT NULL," + @@ -2731,7 +3126,7 @@ func (s *testIntegrationSerialSuite) Test19942(c *C) { tk.MustExec("INSERT INTO test.t (a, b, c, d) VALUES (6, ' E', 'é ', ' E');") mkr := func() [][]interface{} { - return testutil.RowsWithSep("|", + return testkit.RowsWithSep("|", "3| 3 | 3 | 3", "2| 2 0| 2", "5| A | A | A", @@ -2748,8 +3143,10 @@ func (s *testIntegrationSerialSuite) Test19942(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite) TestPartitionUnionWithPPruningColumn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPartitionUnionWithPPruningColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("CREATE TABLE `t` (\n `fid` bigint(36) NOT NULL,\n `oty` varchar(30) DEFAULT NULL,\n `oid` int(11) DEFAULT NULL,\n `pid` bigint(20) DEFAULT NULL,\n `bid` int(11) DEFAULT NULL,\n `r5` varchar(240) DEFAULT '',\n PRIMARY KEY (`fid`)\n)PARTITION BY HASH( `fid` ) PARTITIONS 4;") @@ -2804,8 +3201,10 @@ func (s *testIntegrationSuite) TestPartitionUnionWithPPruningColumn(c *C) { "3290 LE1327_r5")) } -func (s *testIntegrationSuite) TestIssue20139(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20139(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2820,8 +3219,10 @@ func (s *testIntegrationSuite) TestIssue20139(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSuite) TestIssue14481(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue14481(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2831,8 +3232,11 @@ func (s *testIntegrationSuite) TestIssue14481(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSerialSuite) TestIssue20710(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestIssue20710(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("drop table if exists s;") tk.MustExec("create table t(a int, b int)") @@ -2845,30 +3249,35 @@ func (s *testIntegrationSerialSuite) TestIssue20710(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestQueryBlockTableAliasInHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestQueryBlockTableAliasInHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - c.Assert(tk.HasPlan("select /*+ HASH_JOIN(@sel_1 t2) */ * FROM (select 1) t1 NATURAL LEFT JOIN (select 2) t2", "HashJoin"), IsTrue) + require.True(t, tk.HasPlan("select /*+ HASH_JOIN(@sel_1 t2) */ * FROM (select 1) t1 NATURAL LEFT JOIN (select 2) t2", "HashJoin")) tk.MustQuery("select /*+ HASH_JOIN(@sel_1 t2) */ * FROM (select 1) t1 NATURAL LEFT JOIN (select 2) t2").Check(testkit.Rows( "1 2", )) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 0) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 0) } -func (s *testIntegrationSuite) TestIssue10448(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue10448(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") @@ -2877,8 +3286,10 @@ func (s *testIntegrationSuite) TestIssue10448(c *C) { tk.MustQuery("select a from (select pk as a from t) t1 where a = 18446744073709551615").Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestMultiUpdateOnPrimaryKey(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiUpdateOnPrimaryKey(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2923,8 +3334,10 @@ func (s *testIntegrationSuite) TestMultiUpdateOnPrimaryKey(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("11 12")) } -func (s *testIntegrationSuite) TestOrderByHavingNotInSelect(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderByHavingNotInSelect(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists ttest") tk.MustExec("create table ttest (v1 int, v2 int)") @@ -2940,9 +3353,11 @@ func (s *testIntegrationSuite) TestOrderByHavingNotInSelect(c *C) { } -func (s *testIntegrationSuite) TestUpdateSetDefault(c *C) { +func TestUpdateSetDefault(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // #20598 - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table tt (x int, z int as (x+10) stored)") tk.MustExec("insert into tt(x) values (1)") @@ -2962,8 +3377,10 @@ func (s *testIntegrationSuite) TestUpdateSetDefault(c *C) { "[planner:3105]The value specified for generated column 'z' in table 'tt' is not allowed.") } -func (s *testIntegrationSuite) TestExtendedStatsSwitch(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExtendedStatsSwitch(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int not null, b int not null, key(a), key(b))") @@ -3015,8 +3432,10 @@ func (s *testIntegrationSuite) TestExtendedStatsSwitch(c *C) { )) } -func (s *testIntegrationSuite) TestOrderByNotInSelectDistinct(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderByNotInSelectDistinct(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // #12442 @@ -3055,9 +3474,11 @@ func (s *testIntegrationSuite) TestOrderByNotInSelectDistinct(c *C) { tk.MustQuery("select distinct v1 as z from ttest order by v1+z").Check(testkit.Rows("1", "4")) } -func (s *testIntegrationSuite) TestInvalidNamedWindowSpec(c *C) { +func TestInvalidNamedWindowSpec(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // #12356 - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("DROP TABLE IF EXISTS temptest") tk.MustExec("create table temptest (val int, val1 int)") @@ -3070,8 +3491,10 @@ func (s *testIntegrationSuite) TestInvalidNamedWindowSpec(c *C) { "[planner:1054]Unknown column 'a' in 'window partition by'") } -func (s *testIntegrationSuite) TestCorrelatedAggregate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCorrelatedAggregate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // #18350 @@ -3150,8 +3573,10 @@ func (s *testIntegrationSuite) TestCorrelatedAggregate(c *C) { Check(testkit.Rows("6 6 6 6")) } -func (s *testIntegrationSuite) TestCorrelatedColumnAggFuncPushDown(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCorrelatedColumnAggFuncPushDown(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int);") @@ -3162,8 +3587,10 @@ func (s *testIntegrationSuite) TestCorrelatedColumnAggFuncPushDown(c *C) { } // Test for issue https://github.com/pingcap/tidb/issues/21607. -func (s *testIntegrationSuite) TestConditionColPruneInPhysicalUnionScan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestConditionColPruneInPhysicalUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int, b int);") @@ -3182,8 +3609,10 @@ func (s *testIntegrationSuite) TestConditionColPruneInPhysicalUnionScan(c *C) { Check(testkit.Rows("0")) } -func (s *testIntegrationSuite) TestInvalidHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestInvalidHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tt") @@ -3195,21 +3624,24 @@ func (s *testIntegrationSuite) TestInvalidHint(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) warning := "show warnings;" for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery(warning).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery(warning).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } // Test for issue https://github.com/pingcap/tidb/issues/18320 -func (s *testIntegrationSuite) TestNonaggregateColumnWithSingleValueInOnlyFullGroupByMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNonaggregateColumnWithSingleValueInOnlyFullGroupByMode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int)") @@ -3221,8 +3653,10 @@ func (s *testIntegrationSuite) TestNonaggregateColumnWithSingleValueInOnlyFullGr tk.MustQuery("select a from t where a = 1 having count(b) > 0").Check(testkit.Rows("1")) } -func (s *testIntegrationSuite) TestConvertRangeToPoint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestConvertRangeToPoint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0") @@ -3249,19 +3683,22 @@ func (s *testIntegrationSuite) TestConvertRangeToPoint(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIssue22040(c *C) { +func TestIssue22040(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // #22040 - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, primary key(a,b))") @@ -3270,16 +3707,18 @@ func (s *testIntegrationSuite) TestIssue22040(c *C) { // invalid case, column count doesn't match { err := tk.ExecToErr("select * from t where (a,b) in (1,2)") - c.Assert(errors.Cause(err), FitsTypeOf, expression.ErrOperandColumns) + require.IsType(t, expression.ErrOperandColumns, errors.Cause(err)) } { err := tk.ExecToErr("select * from t where (a,b) in ((1,2),1)") - c.Assert(errors.Cause(err), FitsTypeOf, expression.ErrOperandColumns) + require.IsType(t, expression.ErrOperandColumns, errors.Cause(err)) } } -func (s *testIntegrationSuite) TestIssue22105(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22105(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -3305,18 +3744,21 @@ func (s *testIntegrationSuite) TestIssue22105(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIssue22071(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22071(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int);") tk.MustExec("insert into t values(1),(2),(5)") @@ -3324,12 +3766,14 @@ func (s *testIntegrationSuite) TestIssue22071(c *C) { tk.MustQuery("select n in (1,n) from (select a in (1,2) as n from t) g;").Check(testkit.Rows("1", "1", "1")) } -func (s *testIntegrationSuite) TestCreateViewIsolationRead(c *C) { - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue) - tk := testkit.NewTestKit(c, s.store) - tk.Se = se +func TestCreateViewIsolationRead(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk := testkit.NewTestKit(t, store) + tk.SetSession(se) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") @@ -3342,8 +3786,10 @@ func (s *testIntegrationSuite) TestCreateViewIsolationRead(c *C) { tk.MustQuery("select * from v0;").Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestIssue22199(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22199(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(i int primary key, j int, index idx_j(j))") @@ -3351,8 +3797,10 @@ func (s *testIntegrationSuite) TestIssue22199(c *C) { tk.MustGetErrMsg("select t1.*, (select t2.* from t1) from t1", "[planner:1051]Unknown table 't2'") } -func (s *testIntegrationSuite) TestIssue22892(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22892(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("drop table if exists t1") @@ -3367,8 +3815,10 @@ func (s *testIntegrationSuite) TestIssue22892(c *C) { tk.MustQuery("select * from t2 where a not between 1 and 2;").Check(testkit.Rows("0")) } -func (s *testIntegrationSuite) TestIssue26719(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26719(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table tx (a int) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20))`) tk.MustExec(`insert into tx values (1)`) @@ -3381,8 +3831,34 @@ func (s *testIntegrationSuite) TestIssue26719(c *C) { tk.MustExec(`rollback`) } -func (s *testIntegrationSerialSuite) TestPushDownProjectionForTiFlash(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue32428(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table `t1` (`a` enum('aa') DEFAULT NULL, KEY `k` (`a`))") + tk.MustExec("insert into t1 values('aa')") + tk.MustExec("insert into t1 values(null)") + tk.MustQuery("select a from t1 where a<=>'aa'").Check(testkit.Rows("aa")) + tk.MustQuery("select a from t1 where a<=>null").Check(testkit.Rows("")) + + tk.MustExec(`CREATE TABLE IDT_MULTI15860STROBJSTROBJ ( + COL1 enum('aa') DEFAULT NULL, + COL2 int(41) DEFAULT NULL, + COL3 year(4) DEFAULT NULL, + KEY U_M_COL4 (COL1,COL2), + KEY U_M_COL5 (COL3,COL2))`) + tk.MustExec(`insert into IDT_MULTI15860STROBJSTROBJ values("aa", 1013610488, 1982)`) + tk.MustQuery(`SELECT * FROM IDT_MULTI15860STROBJSTROBJ t1 RIGHT JOIN IDT_MULTI15860STROBJSTROBJ t2 ON t1.col1 <=> t2.col1 where t1.col1 is null and t2.col1 = "aa"`).Check(testkit.Rows()) // empty result + tk.MustExec(`prepare stmt from "SELECT * FROM IDT_MULTI15860STROBJSTROBJ t1 RIGHT JOIN IDT_MULTI15860STROBJSTROBJ t2 ON t1.col1 <=> t2.col1 where t1.col1 is null and t2.col1 = ?"`) + tk.MustExec(`set @a="aa"`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows()) // empty result +} + +func TestPushDownProjectionForTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3), name char(128))") @@ -3390,10 +3866,10 @@ func (s *testIntegrationSerialSuite) TestPushDownProjectionForTiFlash(c *C) { tk.MustExec("set session tidb_allow_mpp=OFF") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3410,29 +3886,32 @@ func (s *testIntegrationSerialSuite) TestPushDownProjectionForTiFlash(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestPushDownProjectionForMPP(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPushDownProjectionForMPP(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3), name char(128))") tk.MustExec("analyze table t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3449,19 +3928,22 @@ func (s *testIntegrationSerialSuite) TestPushDownProjectionForMPP(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestReorderSimplifiedOuterJoins(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestReorderSimplifiedOuterJoins(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2,t3") @@ -3474,19 +3956,22 @@ func (s *testIntegrationSuite) TestReorderSimplifiedOuterJoins(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } // Apply operator may got panic because empty Projection is eliminated. -func (s *testIntegrationSerialSuite) TestIssue23887(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23887(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b int);") @@ -3497,12 +3982,13 @@ func (s *testIntegrationSerialSuite) TestIssue23887(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) @@ -3513,8 +3999,10 @@ func (s *testIntegrationSerialSuite) TestIssue23887(c *C) { tk.MustQuery("select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2;").Check(testkit.Rows("1")) } -func (s *testIntegrationSerialSuite) TestDeleteStmt(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDeleteStmt(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int)") tk.MustExec("delete t from t;") @@ -3527,8 +4015,10 @@ func (s *testIntegrationSerialSuite) TestDeleteStmt(c *C) { tk.MustGetErrCode("delete test.t from t;", mysql.ErrUnknownTable) } -func (s *testIntegrationSuite) TestIndexMergeConstantTrue(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeConstantTrue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int primary key, b int not null, key(b))") @@ -3543,18 +4033,20 @@ func (s *testIntegrationSuite) TestIndexMergeConstantTrue(c *C) { tk.MustExec("delete /*+ use_index_merge(t) */ FROM t WHERE a=1 OR (a<2 and b<2)") } -func (s *testIntegrationSerialSuite) TestPushDownAggForMPP(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPushDownAggForMPP(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3))") tk.MustExec("analyze table t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3571,19 +4063,22 @@ func (s *testIntegrationSerialSuite) TestPushDownAggForMPP(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMppUnionAll(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMppUnionAll(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists t1") @@ -3591,10 +4086,10 @@ func (s *testIntegrationSerialSuite) TestMppUnionAll(c *C) { tk.MustExec("create table t1 (a int, b int not null, c double)") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" || tblInfo.Name.L == "t1" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3609,11 +4104,12 @@ func (s *testIntegrationSerialSuite) TestMppUnionAll(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) @@ -3621,8 +4117,10 @@ func (s *testIntegrationSerialSuite) TestMppUnionAll(c *C) { } -func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMppJoinDecimal(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists tt") @@ -3632,10 +4130,10 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { tk.MustExec("analyze table tt") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" || tblInfo.Name.L == "tt" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3654,29 +4152,32 @@ func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestMppAggTopNWithJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMppAggTopNWithJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3))") tk.MustExec("analyze table t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -3693,19 +4194,22 @@ func (s *testIntegrationSerialSuite) TestMppAggTopNWithJoin(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestLimitIndexLookUpKeepOrder(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLimitIndexLookUpKeepOrder(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b int, c int, d int, index idx(a,b,c));") @@ -3715,18 +4219,21 @@ func (s *testIntegrationSerialSuite) TestLimitIndexLookUpKeepOrder(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestDecorrelateInnerJoinInSubquery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestDecorrelateInnerJoinInSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -3737,18 +4244,21 @@ func (s *testIntegrationSuite) TestDecorrelateInnerJoinInSubquery(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIndexMergeTableFilter(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeTableFilter(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int, b int, c int, d int, key(a), key(b));") @@ -3776,16 +4286,20 @@ func (s *testIntegrationSuite) TestIndexMergeTableFilter(c *C) { )) } -func (s *testIntegrationSuite) TestIssue22850(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22850(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("CREATE TABLE t1 (a int(11))") tk.MustQuery("SELECT @v:=(SELECT 1 FROM t1 t2 LEFT JOIN t1 ON t1.a GROUP BY t1.a) FROM t1").Check(testkit.Rows()) // work fine } -func (s *testIntegrationSuite) TestJoinSchemaChange(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestJoinSchemaChange(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int(11))") @@ -3794,8 +4308,10 @@ func (s *testIntegrationSuite) TestJoinSchemaChange(c *C) { } // #22949: test HexLiteral Used in GetVar expr -func (s *testIntegrationSuite) TestGetVarExprWithHexLiteral(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGetVarExprWithHexLiteral(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1_no_idx;") tk.MustExec("create table t1_no_idx(id int, col_bit bit(16));") @@ -3844,8 +4360,10 @@ func (s *testIntegrationSuite) TestGetVarExprWithHexLiteral(c *C) { } // test BitLiteral used with GetVar -func (s *testIntegrationSuite) TestGetVarExprWithBitLiteral(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGetVarExprWithBitLiteral(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1_no_idx;") tk.MustExec("create table t1_no_idx(id int, col_bit bit(16));") @@ -3863,8 +4381,10 @@ func (s *testIntegrationSuite) TestGetVarExprWithBitLiteral(c *C) { tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) } -func (s *testIntegrationSuite) TestIndexMergeClusterIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeClusterIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 float, c2 int, c3 int, primary key (c1) /*T![clustered_index] CLUSTERED */, key idx_1 (c2), key idx_2 (c3))") @@ -3882,8 +4402,10 @@ func (s *testIntegrationSuite) TestIndexMergeClusterIndex(c *C) { )) } -func (s *testIntegrationSuite) TestMultiColMaxOneRow(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMultiColMaxOneRow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") @@ -3895,18 +4417,21 @@ func (s *testIntegrationSuite) TestMultiColMaxOneRow(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIssue23736(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23736(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t0, t1") tk.MustExec("create table t0(a int, b int, c int as (a + b) virtual, unique index (c) invisible);") @@ -3920,26 +4445,30 @@ func (s *testIntegrationSuite) TestIssue23736(c *C) { tk.MustQuery("select /*+ nth_plan(3) */ count(1) from t0 where c > 10 and b < 2;").Check(testkit.Rows("0")) // Should not use invisible index - c.Assert(tk.MustUseIndex("select /*+ stream_agg() */ count(1) from t0 where c > 10 and b < 2", "c"), IsFalse) + require.False(t, tk.MustUseIndex("select /*+ stream_agg() */ count(1) from t0 where c > 10 and b < 2", "c")) } // https://github.com/pingcap/tidb/issues/23802 -func (s *testIntegrationSuite) TestPanicWhileQueryTableWithIsNull(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPanicWhileQueryTableWithIsNull(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists NT_HP27193") tk.MustExec("CREATE TABLE `NT_HP27193` ( `COL1` int(20) DEFAULT NULL, `COL2` varchar(20) DEFAULT NULL, `COL4` datetime DEFAULT NULL, `COL3` bigint(20) DEFAULT NULL, `COL5` float DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH ( `COL1`%`COL3` ) PARTITIONS 10;") _, err := tk.Exec("select col1 from NT_HP27193 where col1 is null;") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("INSERT INTO NT_HP27193 (COL2, COL4, COL3, COL5) VALUES ('m', '2020-05-04 13:15:27', 8, 2602)") _, err = tk.Exec("select col1 from NT_HP27193 where col1 is null;") - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("drop table if exists NT_HP27193") } -func (s *testIntegrationSuite) TestIssue23846(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23846(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a varbinary(10),UNIQUE KEY(a))") @@ -3948,8 +4477,10 @@ func (s *testIntegrationSuite) TestIssue23846(c *C) { tk.MustQuery("select * from t where a=0x00A4EEF4FA55D6706ED5").Check(testkit.Rows("\x00\xa4\xee\xf4\xfaU\xd6pn\xd5")) // not empty } -func (s *testIntegrationSuite) TestIssue23839(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23839(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists BB") tk.MustExec("CREATE TABLE `BB` (\n" + @@ -3968,8 +4499,10 @@ func (s *testIntegrationSuite) TestIssue23839(c *C) { } // https://github.com/pingcap/tidb/issues/24095 -func (s *testIntegrationSuite) TestIssue24095(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24095(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (id int, value decimal(10,5));") @@ -3980,18 +4513,21 @@ func (s *testIntegrationSuite) TestIssue24095(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIssue24281(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue24281(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists member, agent, deposit, view_member_agents") tk.MustExec("create table member(login varchar(50) NOT NULL, agent_login varchar(100) DEFAULT NULL, PRIMARY KEY(login))") @@ -4007,8 +4543,10 @@ func (s *testIntegrationSuite) TestIssue24281(c *C) { "UNION select 1 as v1, 2 as v2") } -func (s *testIntegrationSuite) TestIssue25799(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue25799(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec(`create table t1 (a float default null, b smallint(6) DEFAULT NULL)`) @@ -4019,8 +4557,10 @@ func (s *testIntegrationSuite) TestIssue25799(c *C) { tk.MustQuery(`select /*+ TIDB_INLJ(t2@sel_2) */ t1.a, t1.b from t1 where t1.a not in (select t2.a from t2 where t1.b=t2.b)`).Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestLimitWindowColPrune(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLimitWindowColPrune(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -4028,34 +4568,39 @@ func (s *testIntegrationSuite) TestLimitWindowColPrune(c *C) { tk.MustQuery("select count(a) f1, row_number() over (order by count(a)) as f2 from t limit 1").Check(testkit.Rows("1 1")) } -func (s *testIntegrationSuite) TestIncrementalAnalyzeStatsVer2(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIncrementalAnalyzeStatsVer2(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index idx_b(b))") tk.MustExec("insert into t values(1,1),(2,2),(3,3)") tk.MustExec("set @@session.tidb_analyze_version = 2") tk.MustExec("analyze table t") - is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblID := tbl.Meta().ID rows := tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][0], Equals, "3") + require.Len(t, rows, 1) + require.Equal(t, "3", rows[0][0]) tk.MustExec("insert into t values(4,4),(5,5),(6,6)") tk.MustExec("analyze incremental table t index idx_b") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 3) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "The version 2 would collect all statistics not only the selected indexes") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[1].Err.Error(), Equals, "The version 2 stats would ignore the INCREMENTAL keyword and do full sampling") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[2].Err.Error(), Equals, "Analyze use auto adjusted sample rate 1.000000 for table test.t.") + warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warns, 3) + require.EqualError(t, warns[0].Err, "The version 2 would collect all statistics not only the selected indexes") + require.EqualError(t, warns[1].Err, "The version 2 stats would ignore the INCREMENTAL keyword and do full sampling") + require.EqualError(t, warns[2].Err, "Analyze use auto adjusted sample rate 1.000000 for table test.t.") rows = tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][0], Equals, "6") + require.Len(t, rows, 1) + require.Equal(t, "6", rows[0][0]) } -func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestConflictReadFromStorage(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(`create table t ( @@ -4067,10 +4612,10 @@ func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { partition p2 values less than(16));`) tk.MustExec(`insert into t values (1,1,"1"), (2,2,"2"), (8,8,"8"), (11,11,"11"), (15,15,"15")`) // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -4086,8 +4631,10 @@ func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { } // TestSequenceAsDataSource is used to test https://github.com/pingcap/tidb/issues/24383. -func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSequenceAsDataSource(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop sequence if exists s1, s2") @@ -4099,20 +4646,21 @@ func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestIssue27167(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) +func TestIssue27167(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set names utf8mb4") tk.MustExec("use test") tk.MustExec("drop table if exists all_types") @@ -4146,10 +4694,10 @@ func (s *testIntegrationSerialSuite) TestIssue27167(c *C) { tk.MustQuery("select collation(c) from (select d_timestamp c from all_types union select d_float c from all_types) t").Check(testkit.Rows("utf8mb4_bin", "utf8mb4_bin")) } -func (s *testIntegrationSerialSuite) TestIssue25300(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tk := testkit.NewTestKit(c, s.store) +func TestIssue25300(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`create table t (a char(65) collate utf8_unicode_ci, b text collate utf8_general_ci not null);`) tk.MustExec(`insert into t values ('a', 'A');`) @@ -4161,17 +4709,19 @@ func (s *testIntegrationSerialSuite) TestIssue25300(c *C) { tk.MustGetErrCode(`(select a from t) union ( select b from t) union select 'a' except select 'd';`, mysql.ErrCantAggregateNcollations) } -func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestMergeContinuousSelections(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists ts") tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) not null);") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "ts" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -4188,22 +4738,25 @@ func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestSelectIgnoreTemporaryTableInView(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectIgnoreTemporaryTableInView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) tk.MustExec("create table t1 (a int, b int)") tk.MustExec("create table t2 (c int, d int)") tk.MustExec("create view v1 as select * from t1 order by a") @@ -4229,8 +4782,10 @@ func (s *testIntegrationSerialSuite) TestSelectIgnoreTemporaryTableInView(c *C) } // TestIsMatchProp is used to test https://github.com/pingcap/tidb/issues/26017. -func (s *testIntegrationSuite) TestIsMatchProp(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIsMatchProp(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -4242,18 +4797,21 @@ func (s *testIntegrationSuite) TestIsMatchProp(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestIssue26250(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26250(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table tp (id int primary key) partition by range (id) (partition p0 values less than (100));") tk.MustExec("create table tn (id int primary key);") @@ -4262,8 +4820,10 @@ func (s *testIntegrationSerialSuite) TestIssue26250(c *C) { tk.MustQuery("select * from tp,tn where tp.id=tn.id and tn.id=1 for update;").Check(testkit.Rows("1 1")) } -func (s *testIntegrationSuite) TestCorrelationAdjustment4Limit(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCorrelationAdjustment4Limit(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (pk int primary key auto_increment, year int, c varchar(256), index idx_year(year))") @@ -4315,8 +4875,10 @@ func (s *testIntegrationSuite) TestCorrelationAdjustment4Limit(c *C) { " └─TableFullScan 51.00 cop[tikv] table:t keep order:false")) } -func (s *testIntegrationSerialSuite) TestCTESelfJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCTESelfJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3") tk.MustExec("create table t1(t1a int, t1b int, t1c int)") @@ -4338,17 +4900,21 @@ func (s *testIntegrationSerialSuite) TestCTESelfJoin(c *C) { } // https://github.com/pingcap/tidb/issues/26214 -func (s *testIntegrationSerialSuite) TestIssue26214(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26214(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table `t` (`a` int(11) default null, `b` int(11) default null, `c` int(11) default null, key `expression_index` ((case when `a` < 0 then 1 else 2 end)))") _, err := tk.Exec("select * from t where case when a < 0 then 1 else 2 end <= 1 order by 4;") - c.Assert(core.ErrUnknownColumn.Equal(err), IsTrue) + require.True(t, core.ErrUnknownColumn.Equal(err)) } -func (s *testIntegrationSuite) TestCreateViewWithWindowFunc(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCreateViewWithWindowFunc(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t6;") tk.MustExec("CREATE TABLE t6(t TIME, ts TIMESTAMP);") @@ -4359,8 +4925,10 @@ func (s *testIntegrationSuite) TestCreateViewWithWindowFunc(c *C) { rows.Check(testkit.Rows("1 1")) } -func (s *testIntegrationSuite) TestIssue29834(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29834(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists IDT_MC21814;") tk.MustExec("CREATE TABLE `IDT_MC21814` (`COL1` year(4) DEFAULT NULL,`COL2` year(4) DEFAULT NULL,KEY `U_M_COL` (`COL1`,`COL2`)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") @@ -4371,8 +4939,10 @@ func (s *testIntegrationSuite) TestIssue29834(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The parameter of nth_plan() is out of range.")) } -func (s *testIntegrationSuite) TestIssue29221(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29221(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_index_merge=on;") tk.MustExec("drop table if exists t;") @@ -4415,8 +4985,10 @@ func (s *testIntegrationSuite) TestIssue29221(c *C) { " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:t keep order:false, stats:pseudo")) } -func (s *testIntegrationSerialSuite) TestLimitPushDown(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestLimitPushDown(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -4449,16 +5021,20 @@ func (s *testIntegrationSerialSuite) TestLimitPushDown(c *C) { ` └─TableFullScan 1.00 cop[tikv] table:t keep order:false`)) } -func (s *testIntegrationSuite) TestIssue26559(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26559(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a timestamp, b datetime);") tk.MustExec("insert into t values('2020-07-29 09:07:01', '2020-07-27 16:57:36');") tk.MustQuery("select greatest(a, b) from t union select null;").Sort().Check(testkit.Rows("2020-07-29 09:07:01", "")) } -func (s *testIntegrationSuite) TestIssue29503(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29503(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.Status.RecordQPSbyDB = true @@ -4467,16 +5043,16 @@ func (s *testIntegrationSuite) TestIssue29503(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create table t(a int);") - err := tk.ExecToErr("create binding for select 1 using select 1;") - c.Assert(err, Equals, nil) - err = tk.ExecToErr("create binding for select a from t using select a from t;") - c.Assert(err, Equals, nil) + require.NoError(t, tk.ExecToErr("create binding for select 1 using select 1;")) + require.NoError(t, tk.ExecToErr("create binding for select a from t using select a from t;")) res := tk.MustQuery("show session bindings;") - c.Assert(len(res.Rows()), Equals, 2) + require.Len(t, res.Rows(), 2) } -func (s *testIntegrationSuite) TestHeuristicIndexSelection(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHeuristicIndexSelection(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") @@ -4490,20 +5066,23 @@ func (s *testIntegrationSuite) TestHeuristicIndexSelection(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) - output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) }) tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) } } -func (s *testIntegrationSuite) TestOutputSkylinePruningInfo(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOutputSkylinePruningInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") @@ -4514,26 +5093,29 @@ func (s *testIntegrationSuite) TestOutputSkylinePruningInfo(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) - output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'verbose' " + tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) }) tk.MustQuery("explain format = 'verbose' " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) } } -func (s *testIntegrationSuite) TestPreferRangeScanForUnsignedIntHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPreferRangeScanForUnsignedIntHandle(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int unsigned primary key, b int, c int, index idx_b(b))") tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10,11,12), (13,14,15)") - do, _ := session.GetDomain(s.store) - c.Assert(do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + do, _ := session.GetDomain(store) + require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table t") var input []string @@ -4542,33 +5124,36 @@ func (s *testIntegrationSuite) TestPreferRangeScanForUnsignedIntHandle(c *C) { Plan []string Warnings []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt }) if strings.HasPrefix(tt, "set") { tk.MustExec(tt) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Warnings = s.testData.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warnings = testdata.ConvertRowsToStrings(tk.MustQuery("show warnings").Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery("show warnings").Check(testkit.Rows(output[i].Warnings...)) } } -func (s *testIntegrationSuite) TestIssue27083(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue27083(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c int, index idx_b(b))") tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18)") - do, _ := session.GetDomain(s.store) - c.Assert(do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + do, _ := session.GetDomain(store) + require.Nil(t, do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table t") var input []string @@ -4576,18 +5161,21 @@ func (s *testIntegrationSuite) TestIssue27083(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSuite) TestIssues27130(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssues27130(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -4625,8 +5213,10 @@ func (s *testIntegrationSuite) TestIssues27130(c *C) { )) } -func (s *testIntegrationSuite) TestIssue27242(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue27242(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists UK_MU16407") tk.MustExec("CREATE TABLE UK_MU16407 (COL3 timestamp NULL DEFAULT NULL, UNIQUE KEY U3(COL3));") @@ -4648,8 +5238,10 @@ func verifyTimestampOutOfRange(tk *testkit.TestKit) { tk.MustQuery(`select * from t28424 where t > "1970-1-1 0:0:0"`).Sort().Check(testkit.Rows("1970-01-01 00:00:01]\n[2038-01-19 03:14:07")) } -func (s *testIntegrationSuite) TestIssue28424(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue28424(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t28424, dt28242") @@ -4674,8 +5266,10 @@ func (s *testIntegrationSuite) TestIssue28424(c *C) { "[ 2038-03-19 03:14:08")) } -func (s *testIntegrationSerialSuite) TestTemporaryTableForCte(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestTemporaryTableForCte(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create temporary table tmp1(a int, b int, c int);") @@ -4688,8 +5282,10 @@ func (s *testIntegrationSerialSuite) TestTemporaryTableForCte(c *C) { rows.Check(testkit.Rows("1", "2", "3", "4", "5")) } -func (s *testIntegrationSuite) TestGroupBySetVar(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestGroupBySetVar(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1(c1 int);") @@ -4705,28 +5301,31 @@ func (s *testIntegrationSuite) TestGroupBySetVar(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { res := tk.MustQuery("explain format = 'brief' " + tt) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(res.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(res.Rows()) }) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPushDownGroupConcatToTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists ts") tk.MustExec("create table ts (col_0 char(64), col_1 varchar(64) not null, col_2 varchar(1), id int primary key);") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "ts" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -4744,18 +5343,19 @@ func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { Plan []string Warning []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) - comment := Commentf("case:%v sql:%s", i, tt) - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { if len(warnings) > 0 { output[i].Warning = make([]string, len(warnings)) for j, warning := range warnings { @@ -4764,19 +5364,21 @@ func (s *testIntegrationSerialSuite) TestPushDownGroupConcatToTiFlash(c *C) { } }) if len(output[i].Warning) == 0 { - c.Assert(len(warnings), Equals, 0, comment) + require.Len(t, warnings, 0, comment) } else { - c.Assert(len(warnings), Equals, len(output[i].Warning), comment) + require.Len(t, warnings, len(output[i].Warning), comment) for j, warning := range warnings { - c.Assert(warning.Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warning.Err.Error(), Equals, output[i].Warning[j], comment) + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.EqualError(t, warning.Err, output[i].Warning[j], comment) } } } } -func (s *testIntegrationSuite) TestIssue27797(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue27797(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) origin := tk.MustQuery("SELECT @@session.tidb_partition_prune_mode") originStr := origin.Rows()[0][0].(string) defer func() { @@ -4811,8 +5413,10 @@ func (s *testIntegrationSuite) TestIssue27797(c *C) { result.Check(testkit.Rows("")) } -func (s *testIntegrationSuite) TestIssue27949(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue27949(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t27949") tk.MustExec("create table t27949 (a int, b int, key(b))") @@ -4840,44 +5444,47 @@ func (s *testIntegrationSuite) TestIssue27949(c *C) { tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) } -func (s *testIntegrationSuite) TestIssue28154(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue28154(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") defer func() { - tk.Exec("drop table if exists t") + tk.MustExec("drop table if exists t") }() tk.MustExec("create table t(a TEXT)") tk.MustExec("insert into t values('abc')") result := tk.MustQuery("select * from t where from_base64('')") result.Check(testkit.Rows()) _, err := tk.Exec("update t set a = 'def' where from_base64('')") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[types:1292]Truncated incorrect DOUBLE value: ''") + require.EqualError(t, err, "[types:1292]Truncated incorrect DOUBLE value: ''") result = tk.MustQuery("select * from t where from_base64('invalidbase64')") result.Check(testkit.Rows()) tk.MustExec("update t set a = 'hig' where from_base64('invalidbase64')") result = tk.MustQuery("select * from t where from_base64('test')") result.Check(testkit.Rows()) _, err = tk.Exec("update t set a = 'xyz' where from_base64('test')") - c.Assert(err, NotNil) - c.Assert(err, ErrorMatches, "\\[types:1292\\]Truncated incorrect DOUBLE value.*") + require.Error(t, err) + require.Regexp(t, "\\[types:1292\\]Truncated incorrect DOUBLE value.*", err.Error()) result = tk.MustQuery("select * from t") result.Check(testkit.Rows("abc")) } -func (s *testIntegrationSerialSuite) TestRejectSortForMPP(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRejectSortForMPP(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3), name char(128))") tk.MustExec("analyze table t") // Create virtual tiflash replica info. - dom := domain.GetDomain(tk.Se) + dom := domain.GetDomain(tk.Session()) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -4894,19 +5501,22 @@ func (s *testIntegrationSerialSuite) TestRejectSortForMPP(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) res := tk.MustQuery(tt) res.Check(testkit.Rows(output[i].Plan...)) } } -func (s *testIntegrationSerialSuite) TestRegardNULLAsPoint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRegardNULLAsPoint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tpk") @@ -4934,16 +5544,17 @@ func (s *testIntegrationSerialSuite) TestRegardNULLAsPoint(c *C) { PlanDisabled []string Result []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) - output[i].PlanEnabled = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].PlanEnabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) - output[i].PlanDisabled = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) + output[i].PlanDisabled = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + tt).Rows()) }) tk.MustExec(`set @@session.tidb_regard_null_as_point=true`) tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].PlanEnabled...)) @@ -4955,8 +5566,10 @@ func (s *testIntegrationSerialSuite) TestRegardNULLAsPoint(c *C) { } } -func (s *testIntegrationSuite) TestIssues29711(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssues29711(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tbl_29711") @@ -4990,8 +5603,10 @@ func (s *testIntegrationSuite) TestIssues29711(c *C) { )) } -func (s *testIntegrationSuite) TestIssue27313(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue27313(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a varchar(100), b int, c int, index idx1(a(2), b), index idx2(a))") @@ -5000,8 +5615,10 @@ func (s *testIntegrationSuite) TestIssue27313(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows()) } -func (s *testIntegrationSuite) TestIssue30094(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue30094(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`drop table if exists t30094;`) @@ -5018,8 +5635,10 @@ func (s *testIntegrationSuite) TestIssue30094(c *C) { )) } -func (s *testIntegrationSuite) TestIssue30200(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue30200(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") @@ -5050,20 +5669,23 @@ func (s *testIntegrationSuite) TestIssue30200(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) } } -func (s *testIntegrationSuite) TestIssue29705(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue29705(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) origin := tk.MustQuery("SELECT @@session.tidb_partition_prune_mode") originStr := origin.Rows()[0][0].(string) defer func() { @@ -5078,10 +5700,10 @@ func (s *testIntegrationSuite) TestIssue29705(c *C) { result.Check(testkit.Rows("1")) } -func (s *testIntegrationSerialSuite) TestIssue30271(c *C) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - tk := testkit.NewTestKit(c, s.store) +func TestIssue30271(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a char(10), b char(10), c char(10), index (a, b, c)) collate utf8mb4_bin;") @@ -5091,8 +5713,10 @@ func (s *testIntegrationSerialSuite) TestIssue30271(c *C) { } -func (s *testIntegrationSuite) TestIssue30804(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue30804(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") @@ -5101,12 +5725,14 @@ func (s *testIntegrationSuite) TestIssue30804(c *C) { tk.MustExec("select avg(0) over w from t1 window w as (order by (select 1))") // named window cannot be used in subquery err := tk.ExecToErr("select avg(0) over w from t1 where b > (select sum(t2.a) over w from t2) window w as (partition by t1.b)") - c.Assert(core.ErrWindowNoSuchWindow.Equal(err), IsTrue) + require.True(t, core.ErrWindowNoSuchWindow.Equal(err)) tk.MustExec("select avg(0) over w1 from t1 where b > (select sum(t2.a) over w2 from t2 window w2 as (partition by t2.b)) window w1 as (partition by t1.b)") } -func (s *testIntegrationSuite) TestIndexMergeWarning(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeWarning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -5128,8 +5754,10 @@ func (s *testIntegrationSuite) TestIndexMergeWarning(c *C) { tk.MustQuery("show warnings").Check(testkit.Rows(warningMsg)) } -func (s *testIntegrationSuite) TestIndexMergeWithCorrelatedColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIndexMergeWithCorrelatedColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t1, t2;") @@ -5160,12 +5788,13 @@ func (s *testIntegrationSuite) TestIndexMergeWithCorrelatedColumns(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format=brief " + tt).Rows()) + output[i].Res = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery("explain format=brief " + tt).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) @@ -5173,8 +5802,10 @@ func (s *testIntegrationSuite) TestIndexMergeWithCorrelatedColumns(c *C) { } -func (s *testIntegrationSuite) TestIssue20510(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20510(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -5199,8 +5830,10 @@ func (s *testIntegrationSuite) TestIssue20510(c *C) { )) } -func (s *testIntegrationSuite) TestIssue31035(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue31035(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1(c1 longtext, c2 decimal(37, 4), unique key(c1(10)), unique key(c2));") @@ -5210,8 +5843,10 @@ func (s *testIntegrationSuite) TestIssue31035(c *C) { // TestDNFCondSelectivityWithConst test selectivity calculation with DNF conditions with one is const. // Close https://github.com/pingcap/tidb/issues/31096 -func (s *testIntegrationSuite) TestDNFCondSelectivityWithConst(c *C) { - testKit := testkit.NewTestKit(c, s.store) +func TestDNFCondSelectivityWithConst(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t1") testKit.MustExec("create table t1(a int, b int, c int);") @@ -5253,15 +5888,16 @@ func (s *testIntegrationSuite) TestDNFCondSelectivityWithConst(c *C) { testKit.MustExec("drop table if exists t1") } -func (s *testIntegrationSuite) TestIssue31202(c *C) { - store, dom := s.store, s.dom - tk := testkit.NewTestKit(c, store) +func TestIssue31202(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t31202(a int primary key, b int);") tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31202", L: "t31202"}) - c.Assert(err, IsNil) + require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -5274,3 +5910,111 @@ func (s *testIntegrationSuite) TestIssue31202(c *C) { "└─TableFullScan 10000.00 cop[tikv] table:t31202 keep order:false, stats:pseudo")) tk.MustExec("drop table if exists t31202") } + +func TestNaturalJoinUpdateSameTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database natural_join_update") + defer tk.MustExec("drop database natural_join_update") + tk.MustExec("use natural_join_update") + tk.MustExec("create table t1(a int, b int)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustExec("update t1 as a natural join t1 b SET a.a = 2, b.b = 3") + tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2 3", "2 3")) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (a int primary key, b int)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 b SET a.a = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (a int, b int) partition by hash (a) partitions 3") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 b SET a.a = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (A int, b int) partition by hash (b) partitions 3") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + _, err := tk.Exec(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`) + require.Error(t, err) + require.Regexp(t, ".planner:1706.Primary key/partition key update is not allowed since the table is updated both as 'a' and 'B'.", err.Error()) + tk.MustExec("drop table t1") + tk.MustExec("create table t1 (A int, b int) partition by RANGE COLUMNS (b) (partition `pNeg` values less than (0),partition `pPos` values less than MAXVALUE)") + tk.MustExec("insert into t1 values (1,1),(2,2)") + tk.MustGetErrCode(`update t1 as a natural join t1 B SET a.A = 2, b.b = 3`, mysql.ErrMultiUpdateKeyConflict) + tk.MustExec("drop table t1") +} + +func TestAggPushToCopForCachedTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec(`create table t32157( + process_code varchar(8) NOT NULL, + ctrl_class varchar(2) NOT NULL, + ctrl_type varchar(1) NOT NULL, + oper_no varchar(12) DEFAULT NULL, + modify_date datetime DEFAULT NULL, + d_c_flag varchar(2) NOT NULL, + PRIMARY KEY (process_code,ctrl_class,d_c_flag));`) + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '1', '10000', '2016-06-29 00:00:00', 'C')") + tk.MustExec("insert into t32157 values ('GDEP0071', '05', '0', '0000', '2016-06-01 00:00:00', 'D')") + tk.MustExec("alter table t32157 cache") + + tk.MustQuery("explain format = 'brief' select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows( + "StreamAgg 1.00 root funcs:count(1)->Column#8]\n" + + "[└─TableReader 10.00 root data:Selection]\n" + + "[ └─Selection 10.00 cop[tikv] eq(test.t32157.process_code, \"GDEP0071\")]\n" + + "[ └─TableFullScan 10000.00 cop[tikv] table:t32157 keep order:false, stats:pseudo")) + + var readFromCacheNoPanic bool + for i := 0; i < 10; i++ { + tk.MustQuery("select /*+AGG_TO_COP()*/ count(*) from t32157 ignore index(primary) where process_code = 'GDEP0071'").Check(testkit.Rows("2")) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + readFromCacheNoPanic = true + break + } + } + require.True(t, readFromCacheNoPanic) + + tk.MustExec("drop table if exists t31202") +} + +func TestIssue31240(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t31240(a int, b int);") + tk.MustExec("set @@tidb_allow_mpp = 0") + + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t31240", L: "t31240"}) + require.NoError(t, err) + // Set the hacked TiFlash replica for explain tests. + tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} + + var input []string + var output []struct { + SQL string + Plan []string + } + integrationSuiteData := core.GetIntegrationSuiteData() + integrationSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MustExec("drop table if exists t31240") +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 13b8e87c9c022..a73759eb09961 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1800,7 +1800,7 @@ func getUintFromNode(ctx sessionctx.Context, n ast.Node) (uVal uint64, isNull bo if !v.InExecute { return 0, false, true } - param, err := expression.ParamMarkerExpression(ctx, v) + param, err := expression.ParamMarkerExpression(ctx, v, false) if err != nil { return 0, false, false } @@ -4165,6 +4165,20 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as ds.SampleInfo = NewTableSampleInfo(tn.TableSample, schema.Clone(), b.partitionedTable) b.isSampling = ds.SampleInfo != nil + for i, colExpr := range ds.Schema().Columns { + var expr expression.Expression + if i < len(columns) { + if columns[i].IsGenerated() && !columns[i].GeneratedStored { + var err error + expr, _, err = b.rewrite(ctx, columns[i].GeneratedExpr, ds, nil, true) + if err != nil { + return nil, err + } + colExpr.VirtualExpr = expr.Clone() + } + } + } + // Init commonHandleCols and commonHandleLens for data source. if tableInfo.IsCommonHandle { ds.commonHandleCols, ds.commonHandleLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, tables.FindPrimaryIndex(tableInfo)) @@ -4173,6 +4187,21 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as for _, path := range ds.possibleAccessPaths { if !path.IsIntHandlePath { path.FullIdxCols, path.FullIdxColLens = expression.IndexInfo2Cols(ds.Columns, ds.schema.Columns, path.Index) + + // check whether the path's index has a tidb_shard() prefix and the index column count + // more than 1. e.g. index(tidb_shard(a), a) + // set UkShardIndexPath only for unique secondary index + if !path.IsCommonHandlePath { + // tidb_shard expression must be first column of index + col := path.FullIdxCols[0] + if col != nil && + expression.GcColumnExprIsTidbShard(col.VirtualExpr) && + len(path.Index.Columns) > 1 && + path.Index.Unique { + path.IsUkShardIndexPath = true + ds.containExprPrefixUk = true + } + } } } @@ -4183,49 +4212,12 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as us.SetChildren(ds) result = us } - // If a table is a cache table, it is judged whether it satisfies the conditions of read cache. - if tableInfo.TableCacheStatusType == model.TableCacheStatusEnable && b.ctx.GetSessionVars().SnapshotTS == 0 && !b.ctx.GetSessionVars().StmtCtx.IsStaleness { - cachedTable := tbl.(table.CachedTable) - txn, err := b.ctx.Txn(true) - if err != nil { - return nil, err - } - leaseDuration := time.Duration(variable.TableCacheLease.Load()) * time.Second - // Use the TS of the transaction to determine whether the cache can be used. - cacheData := cachedTable.TryReadFromCache(txn.StartTS(), leaseDuration) - if cacheData != nil { - sessionVars.StmtCtx.ReadFromTableCache = true - us := LogicalUnionScan{handleCols: handleCols, cacheTable: cacheData}.Init(b.ctx, b.getSelectOffset()) - us.SetChildren(ds) - result = us - } else { - if !b.inUpdateStmt && !b.inDeleteStmt && !sessionVars.StmtCtx.InExplainStmt { - startTS := txn.StartTS() - store := b.ctx.GetStore() - cachedTable.UpdateLockForRead(ctx, store, startTS, leaseDuration) - } - } - } if sessionVars.StmtCtx.TblInfo2UnionScan == nil { sessionVars.StmtCtx.TblInfo2UnionScan = make(map[*model.TableInfo]bool) } sessionVars.StmtCtx.TblInfo2UnionScan[tableInfo] = dirty - for i, colExpr := range ds.Schema().Columns { - var expr expression.Expression - if i < len(columns) { - if columns[i].IsGenerated() && !columns[i].GeneratedStored { - var err error - expr, _, err = b.rewrite(ctx, columns[i].GeneratedExpr, ds, nil, true) - if err != nil { - return nil, err - } - colExpr.VirtualExpr = expr.Clone() - } - } - } - return result, nil } @@ -4783,8 +4775,9 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( } type tblUpdateInfo struct { - name string - pkUpdated bool + name string + pkUpdated bool + partitionColUpdated bool } // CheckUpdateList checks all related columns in updatable state. @@ -4793,7 +4786,12 @@ func CheckUpdateList(assignFlags []int, updt *Update, newTblID2Table map[int64]t for _, content := range updt.TblColPosInfos { tbl := newTblID2Table[content.TblID] flags := assignFlags[content.Start:content.End] - var update, updatePK bool + var update, updatePK, updatePartitionCol bool + var partitionColumnNames []model.CIStr + if pt, ok := tbl.(table.PartitionedTable); ok && pt != nil { + partitionColumnNames = pt.GetPartitionColumnNames() + } + for i, col := range tbl.WritableCols() { if flags[i] >= 0 && col.State != model.StatePublic { return ErrUnknownColumn.GenWithStackByArgs(col.Name, clauseMsg[fieldList]) @@ -4803,19 +4801,25 @@ func CheckUpdateList(assignFlags []int, updt *Update, newTblID2Table map[int64]t if mysql.HasPriKeyFlag(col.Flag) { updatePK = true } + for _, partColName := range partitionColumnNames { + if col.Name.L == partColName.L { + updatePartitionCol = true + } + } } } if update { // Check for multi-updates on primary key, // see https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html#error_er_multi_update_key_conflict if otherTable, ok := updateFromOtherAlias[tbl.Meta().ID]; ok { - if otherTable.pkUpdated || updatePK { + if otherTable.pkUpdated || updatePK || otherTable.partitionColUpdated || updatePartitionCol { return ErrMultiUpdateKeyConflict.GenWithStackByArgs(otherTable.name, updt.names[content.Start].TblName.O) } } else { updateFromOtherAlias[tbl.Meta().ID] = tblUpdateInfo{ - name: updt.names[content.Start].TblName.O, - pkUpdated: updatePK, + name: updt.names[content.Start].TblName.O, + pkUpdated: updatePK, + partitionColUpdated: updatePartitionCol, } } } @@ -5025,9 +5029,9 @@ func IsDefaultExprSameColumn(names types.NameSlice, node ast.ExprNode) bool { return false } -func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) (Plan, error) { +func (b *PlanBuilder) buildDelete(ctx context.Context, ds *ast.DeleteStmt) (Plan, error) { b.pushSelectOffset(0) - b.pushTableHints(delete.TableHints, 0) + b.pushTableHints(ds.TableHints, 0) defer func() { b.popSelectOffset() // table hints are only visible in the current DELETE statement. @@ -5037,18 +5041,18 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( b.inDeleteStmt = true b.isForUpdateRead = true - if delete.With != nil { + if ds.With != nil { l := len(b.outerCTEs) defer func() { b.outerCTEs = b.outerCTEs[:l] }() - err := b.buildWith(ctx, delete.With) + err := b.buildWith(ctx, ds.With) if err != nil { return nil, err } } - p, err := b.buildResultSetNode(ctx, delete.TableRefs.TableRefs) + p, err := b.buildResultSetNode(ctx, ds.TableRefs.TableRefs) if err != nil { return nil, err } @@ -5056,14 +5060,14 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( oldLen := oldSchema.Len() // For explicit column usage, should use the all-public columns. - if delete.Where != nil { - p, err = b.buildSelection(ctx, p, delete.Where, nil) + if ds.Where != nil { + p, err = b.buildSelection(ctx, p, ds.Where, nil) if err != nil { return nil, err } } if b.ctx.GetSessionVars().TxnCtx.IsPessimistic { - if !delete.IsMultiTable { + if !ds.IsMultiTable { p, err = b.buildSelectLock(p, &ast.SelectLockInfo{ LockType: ast.SelectLockForUpdate, }) @@ -5073,22 +5077,22 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( } } - if delete.Order != nil { - p, err = b.buildSort(ctx, p, delete.Order.Items, nil, nil) + if ds.Order != nil { + p, err = b.buildSort(ctx, p, ds.Order.Items, nil, nil) if err != nil { return nil, err } } - if delete.Limit != nil { - p, err = b.buildLimit(p, delete.Limit) + if ds.Limit != nil { + p, err = b.buildLimit(p, ds.Limit) if err != nil { return nil, err } } // If the delete is non-qualified it does not require Select Priv - if delete.Where == nil && delete.Order == nil { + if ds.Where == nil && ds.Order == nil { b.popVisitInfo() } var authErr error @@ -5116,7 +5120,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( } del := Delete{ - IsMultiTable: delete.IsMultiTable, + IsMultiTable: ds.IsMultiTable, }.Init(b.ctx) del.names = p.OutputNames() @@ -5131,12 +5135,12 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( } // Collect visitInfo. - if delete.Tables != nil { + if ds.Tables != nil { // Delete a, b from a, b, c, d... add a and b. updatableList := make(map[string]bool) tbInfoList := make(map[string]*ast.TableName) - collectTableName(delete.TableRefs.TableRefs, &updatableList, &tbInfoList) - for _, tn := range delete.Tables.Tables { + collectTableName(ds.TableRefs.TableRefs, &updatableList, &tbInfoList) + for _, tn := range ds.Tables.Tables { var canUpdate, foundMatch = false, false name := tn.Name.L if tn.Schema.L == "" { @@ -5176,7 +5180,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( } else { // Delete from a, b, c, d. var tableList []*ast.TableName - tableList = extractTableList(delete.TableRefs.TableRefs, tableList, false) + tableList = extractTableList(ds.TableRefs.TableRefs, tableList, false) for _, v := range tableList { if isCTE(v) { return nil, ErrNonUpdatableTable.GenWithStackByArgs(v.Name.O, "DELETE") @@ -5202,8 +5206,8 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( // Table ID may not be unique for deleting multiple tables, for statements like // `delete from t as t1, t as t2`, the same table has two alias, we have to identify a table // by its alias instead of ID. - tblID2TableName := make(map[int64][]*ast.TableName, len(delete.Tables.Tables)) - for _, tn := range delete.Tables.Tables { + tblID2TableName := make(map[int64][]*ast.TableName, len(ds.Tables.Tables)) + for _, tn := range ds.Tables.Tables { tblID2TableName[tn.TableInfo.ID] = append(tblID2TableName[tn.TableInfo.ID], tn) } tblID2Handle = del.cleanTblID2HandleMap(tblID2TableName, tblID2Handle, del.names) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index aec01a73e8442..efd701c31febe 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -2087,6 +2087,26 @@ func (s *testPlanSuite) TestFastPathInvalidBatchPointGet(c *C) { } } +func (s *testPlanSuite) TestTraceFastPlan(c *C) { + defer testleak.AfterTest(c)() + s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = true + defer func() { + s.ctx.GetSessionVars().StmtCtx.EnableOptimizeTrace = false + }() + s.ctx.GetSessionVars().SnapshotInfoschema = s.is + sql := "select * from t where a=1" + comment := Commentf("sql:%s", sql) + stmt, err := s.ParseOneStmt(sql, "", "") + c.Assert(err, IsNil, comment) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + c.Assert(err, IsNil, comment) + plan := TryFastPlan(s.ctx, stmt) + c.Assert(plan, NotNil) + c.Assert(s.ctx.GetSessionVars().StmtCtx.OptimizeTracer, NotNil) + c.Assert(s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.FinalPlan, NotNil) + c.Assert(s.ctx.GetSessionVars().StmtCtx.OptimizeTracer.IsFastPlan, IsTrue) +} + func (s *testPlanSuite) TestWindowLogicalPlanAmbiguous(c *C) { sql := "select a, max(a) over(), sum(a) over() from t" var planString string diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 79eb82a76b47a..e7c5dde77c110 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -75,12 +75,18 @@ const ( AntiLeftOuterSemiJoin ) -// IsOuterJoin returns if this joiner is a outer joiner +// IsOuterJoin returns if this joiner is an outer joiner func (tp JoinType) IsOuterJoin() bool { return tp == LeftOuterJoin || tp == RightOuterJoin || tp == LeftOuterSemiJoin || tp == AntiLeftOuterSemiJoin } +// IsSemiJoin returns if this joiner is a semi/anti-semi joiner +func (tp JoinType) IsSemiJoin() bool { + return tp == SemiJoin || tp == AntiSemiJoin || + tp == LeftOuterSemiJoin || tp == AntiLeftOuterSemiJoin +} + func (tp JoinType) String() string { switch tp { case InnerJoin: @@ -586,6 +592,10 @@ type DataSource struct { // 1. use `inside insert`, `update`, `delete` or `select for update` statement // 2. isolation level is RC isForUpdateRead bool + + // contain unique index and the first field is tidb_shard(), + // such as (tidb_shard(a), a ...), the fields are more than 2 + containExprPrefixUk bool } // ExtractCorrelatedCols implements LogicalPlan interface. diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index 69db1e9f8a839..16b5e05be28d1 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -16,42 +16,30 @@ package core import ( "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/util" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testUnitTestSuit{}) - -type testUnitTestSuit struct { - ctx sessionctx.Context -} - -func (s *testUnitTestSuit) SetUpSuite(c *C) { - s.ctx = MockContext() -} - -func (s *testUnitTestSuit) newTypeWithFlen(typeByte byte, flen int) *types.FieldType { +func newTypeWithFlen(typeByte byte, flen int) *types.FieldType { tp := types.NewFieldType(typeByte) tp.Flen = flen return tp } -func (s *testUnitTestSuit) SubstituteCol2CorCol(expr expression.Expression, colIDs map[int64]struct{}) (expression.Expression, error) { +func SubstituteCol2CorCol(expr expression.Expression, colIDs map[int64]struct{}) (expression.Expression, error) { switch x := expr.(type) { case *expression.ScalarFunction: newArgs := make([]expression.Expression, 0, len(x.GetArgs())) for _, arg := range x.GetArgs() { - newArg, err := s.SubstituteCol2CorCol(arg, colIDs) + newArg, err := SubstituteCol2CorCol(arg, colIDs) if err != nil { return nil, errors.Trace(err) } @@ -67,8 +55,8 @@ func (s *testUnitTestSuit) SubstituteCol2CorCol(expr expression.Expression, colI return expr, nil } -func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { - defer testleak.AfterTest(c)() +func TestIndexPathSplitCorColCond(t *testing.T) { + ctx := MockContext() totalSchema := expression.NewSchema() totalSchema.Append(&expression.Column{ UniqueID: 1, @@ -80,11 +68,11 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { }) totalSchema.Append(&expression.Column{ UniqueID: 3, - RetType: s.newTypeWithFlen(mysql.TypeVarchar, 10), + RetType: newTypeWithFlen(mysql.TypeVarchar, 10), }) totalSchema.Append(&expression.Column{ UniqueID: 4, - RetType: s.newTypeWithFlen(mysql.TypeVarchar, 10), + RetType: newTypeWithFlen(mysql.TypeVarchar, 10), }) totalSchema.Append(&expression.Column{ UniqueID: 5, @@ -177,11 +165,10 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { remained: "[]", }, } - collate.SetNewCollationEnabledForTest(true) for _, tt := range testCases { - comment := Commentf("failed at case:\nexpr: %v\ncorColIDs: %v\nidxColIDs: %v\nidxColLens: %v\naccess: %v\nremained: %v\n", tt.expr, tt.corColIDs, tt.idxColIDs, tt.idxColLens, tt.access, tt.remained) - filters, err := expression.ParseSimpleExprsWithNames(s.ctx, tt.expr, totalSchema, names) - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("failed at case:\nexpr: %v\ncorColIDs: %v\nidxColIDs: %v\nidxColLens: %v\naccess: %v\nremained: %v\n", tt.expr, tt.corColIDs, tt.idxColIDs, tt.idxColLens, tt.access, tt.remained) + filters, err := expression.ParseSimpleExprsWithNames(ctx, tt.expr, totalSchema, names) + require.NoError(t, err, comment) if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { filters = expression.FlattenCNFConditions(sf) } @@ -191,8 +178,8 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { idMap[id] = struct{}{} } for _, filter := range filters { - trueFilter, err := s.SubstituteCol2CorCol(filter, idMap) - c.Assert(err, IsNil, comment) + trueFilter, err := SubstituteCol2CorCol(filter, idMap) + require.NoError(t, err, comment) trueFilters = append(trueFilters, trueFilter) } path := util.AccessPath{ @@ -202,9 +189,8 @@ func (s *testUnitTestSuit) TestIndexPathSplitCorColCond(c *C) { IdxColLens: tt.idxColLens, } - access, remained := path.SplitCorColAccessCondFromFilters(s.ctx, path.EqCondCount) - c.Assert(fmt.Sprintf("%s", access), Equals, tt.access, comment) - c.Assert(fmt.Sprintf("%s", remained), Equals, tt.remained, comment) + access, remained := path.SplitCorColAccessCondFromFilters(ctx, path.EqCondCount) + require.Equal(t, tt.access, fmt.Sprintf("%s", access), comment) + require.Equal(t, tt.remained, fmt.Sprintf("%s", remained), comment) } - collate.SetNewCollationEnabledForTest(false) } diff --git a/planner/core/main_test.go b/planner/core/main_test.go index c0fb896a5bfd2..5baa8dc7aa9f4 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -24,7 +24,7 @@ import ( "go.uber.org/goleak" ) -var testDataMap = make(testdata.BookKeeper, 2) +var testDataMap = make(testdata.BookKeeper) var indexMergeSuiteData testdata.TestData func TestMain(m *testing.M) { @@ -34,10 +34,21 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "integration_partition_suite") testDataMap.LoadTestSuiteData("testdata", "index_merge_suite") + testDataMap.LoadTestSuiteData("testdata", "plan_normalized_suite") + testDataMap.LoadTestSuiteData("testdata", "stats_suite") + testDataMap.LoadTestSuiteData("testdata", "ordered_result_mode_suite") + testDataMap.LoadTestSuiteData("testdata", "point_get_plan") + testDataMap.LoadTestSuiteData("testdata", "enforce_mpp_suite") + testDataMap.LoadTestSuiteData("testdata", "expression_rewriter_suite") + testDataMap.LoadTestSuiteData("testdata", "partition_pruner") + testDataMap.LoadTestSuiteData("testdata", "plan_suite") + testDataMap.LoadTestSuiteData("testdata", "integration_suite") + testDataMap.LoadTestSuiteData("testdata", "analyze_suite") + indexMergeSuiteData = testDataMap["index_merge_suite"] opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } @@ -52,3 +63,43 @@ func TestMain(m *testing.M) { func GetIntegrationPartitionSuiteData() testdata.TestData { return testDataMap["integration_partition_suite"] } + +func GetPlanNormalizedSuiteData() testdata.TestData { + return testDataMap["plan_normalized_suite"] +} + +func GetStatsSuiteData() testdata.TestData { + return testDataMap["stats_suite"] +} + +func GetOrderedResultModeSuiteData() testdata.TestData { + return testDataMap["ordered_result_mode_suite"] +} + +func GetPointGetPlanData() testdata.TestData { + return testDataMap["point_get_plan"] +} + +func GetEnforceMPPSuiteData() testdata.TestData { + return testDataMap["enforce_mpp_suite"] +} + +func GetExpressionRewriterSuiteData() testdata.TestData { + return testDataMap["expression_rewriter_suite"] +} + +func GetPartitionPrunerData() testdata.TestData { + return testDataMap["partition_pruner"] +} + +func GetPlanSuiteData() testdata.TestData { + return testDataMap["plan_suite"] +} + +func GetIntegrationSuiteData() testdata.TestData { + return testDataMap["integration_suite"] +} + +func GetAnalyzeSuiteData() testdata.TestData { + return testDataMap["analyze_suite"] +} diff --git a/planner/core/memtable_predicate_extractor_test.go b/planner/core/memtable_predicate_extractor_test.go index 663da2cfb92d2..40ab99e289111 100644 --- a/planner/core/memtable_predicate_extractor_test.go +++ b/planner/core/memtable_predicate_extractor_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/set" - "github.com/pingcap/tidb/util/testutil" "github.com/stretchr/testify/require" ) @@ -1661,6 +1660,7 @@ func TestPredicateQuery(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(id int, abclmn int);") + tk.MustExec("create table abclmn(a int);") tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'test' and column_name like 'i%'").Check(testkit.Rows("t")) tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'TEST' and column_name like 'I%'").Check(testkit.Rows("t")) tk.MustQuery("select TABLE_NAME from information_schema.columns where table_schema = 'TEST' and column_name like 'ID'").Check(testkit.Rows("t")) @@ -1673,24 +1673,33 @@ func TestPredicateQuery(t *testing.T) { tk.MustGetErrCode("describe t 'I%'", errno.ErrParse) tk.MustGetErrCode("describe t I%", errno.ErrParse) - tk.MustQuery("show columns from t like 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t like 'ABCLMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t like 'abc%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t like 'ABC%'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show fields in t like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show fields in t like '%LMN'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'abclmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'ABCLMN'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'abc%'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like 'ABC%'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like '%lmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t like '%LMN'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t like '%lmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t like '%LMN'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t like '%lmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t like '%LMN'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t where field like '%lmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show columns in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show fields from t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) - tk.MustQuery("show fields in t where field = 'abclmn'").Check(testutil.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t where field like '%lmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns from t where field = 'abclmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show columns in t where field = 'abclmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields from t where field = 'abclmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) + tk.MustQuery("show fields in t where field = 'abclmn'").Check(testkit.RowsWithSep(",", "abclmn,int(11),YES,,,")) tk.MustQuery("explain t").Check(testkit.Rows("id int(11) YES ", "abclmn int(11) YES ")) tk.MustGetErrCode("show columns from t like id", errno.ErrBadField) tk.MustGetErrCode("show columns from t like `id`", errno.ErrBadField) + + tk.MustQuery("show tables like 't'").Check(testkit.Rows("t")) + tk.MustQuery("show tables like 'T'").Check(testkit.Rows("t")) + tk.MustQuery("show tables like 'ABCLMN'").Check(testkit.Rows("abclmn")) + tk.MustQuery("show tables like 'ABC%'").Check(testkit.Rows("abclmn")) + tk.MustQuery("show tables like '%lmn'").Check(testkit.Rows("abclmn")) + tk.MustQuery("show full tables like '%lmn'").Check(testkit.Rows("abclmn BASE TABLE")) + tk.MustGetErrCode("show tables like T", errno.ErrBadField) + tk.MustGetErrCode("show tables like `T`", errno.ErrBadField) } diff --git a/planner/core/optimizer_test.go b/planner/core/optimizer_test.go index bbcf0055eb4ea..0339498b75a55 100644 --- a/planner/core/optimizer_test.go +++ b/planner/core/optimizer_test.go @@ -15,21 +15,18 @@ package core import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) // LogicalOptimize exports the `logicalOptimize` function for test packages and // doesn't affect the normal package and access control of Golang (tricky ^_^) var LogicalOptimize = logicalOptimize -var _ = Suite(&testPlannerFunctionSuite{}) - -type testPlannerFunctionSuite struct { -} - -func testDecimalConvert(lDec, lLen, rDec, rLen int, lConvert, rConvert bool, cDec, cLen int, c *C) { +func testDecimalConvert(t *testing.T, lDec, lLen, rDec, rLen int, lConvert, rConvert bool, cDec, cLen int) { lType := types.NewFieldType(mysql.TypeNewDecimal) lType.Decimal = lDec lType.Flen = lLen @@ -39,39 +36,40 @@ func testDecimalConvert(lDec, lLen, rDec, rLen int, lConvert, rConvert bool, cDe rType.Flen = rLen cType, lCon, rCon := negotiateCommonType(lType, rType) - c.Assert(cType.Tp, Equals, mysql.TypeNewDecimal) - c.Assert(cType.Decimal, Equals, cDec) - c.Assert(cType.Flen, Equals, cLen) - c.Assert(lConvert, Equals, lCon) - c.Assert(rConvert, Equals, rCon) + require.Equal(t, mysql.TypeNewDecimal, cType.Tp) + require.Equal(t, cDec, cType.Decimal) + require.Equal(t, cLen, cType.Flen) + require.Equal(t, lConvert, lCon) + require.Equal(t, rConvert, rCon) } -func (t *testPlannerFunctionSuite) TestMPPDecimalConvert(c *C) { - testDecimalConvert(5, 9, 5, 8, false, false, 5, 9, c) - testDecimalConvert(5, 8, 5, 9, false, false, 5, 9, c) - testDecimalConvert(0, 8, 0, 11, true, false, 0, 11, c) - testDecimalConvert(0, 16, 0, 11, false, false, 0, 16, c) - testDecimalConvert(5, 9, 4, 9, true, true, 5, 10, c) - testDecimalConvert(5, 8, 4, 9, true, true, 5, 10, c) - testDecimalConvert(5, 9, 4, 8, false, true, 5, 9, c) - testDecimalConvert(10, 16, 0, 11, true, true, 10, 21, c) - testDecimalConvert(5, 19, 0, 20, false, true, 5, 25, c) - testDecimalConvert(20, 20, 0, 60, true, true, 20, 65, c) - testDecimalConvert(20, 40, 0, 60, false, true, 20, 65, c) - testDecimalConvert(0, 40, 0, 60, false, false, 0, 60, c) +func TestMPPDecimalConvert(t *testing.T) { + testDecimalConvert(t, 5, 9, 5, 8, false, false, 5, 9) + testDecimalConvert(t, 5, 8, 5, 9, false, false, 5, 9) + testDecimalConvert(t, 0, 8, 0, 11, true, false, 0, 11) + testDecimalConvert(t, 0, 16, 0, 11, false, false, 0, 16) + testDecimalConvert(t, 5, 9, 4, 9, true, true, 5, 10) + testDecimalConvert(t, 5, 8, 4, 9, true, true, 5, 10) + testDecimalConvert(t, 5, 9, 4, 8, false, true, 5, 9) + testDecimalConvert(t, 10, 16, 0, 11, true, true, 10, 21) + testDecimalConvert(t, 5, 19, 0, 20, false, true, 5, 25) + testDecimalConvert(t, 20, 20, 0, 60, true, true, 20, 65) + testDecimalConvert(t, 20, 40, 0, 60, false, true, 20, 65) + testDecimalConvert(t, 0, 40, 0, 60, false, false, 0, 60) } -func testJoinKeyTypeConvert(leftType, rightType, retType *types.FieldType, lConvert, rConvert bool, c *C) { +func testJoinKeyTypeConvert(t *testing.T, leftType, rightType, retType *types.FieldType, lConvert, rConvert bool) { cType, lCon, rCon := negotiateCommonType(leftType, rightType) - c.Assert(cType.Tp, Equals, retType.Tp) - c.Assert(cType.Flen, Equals, retType.Flen) - c.Assert(cType.Decimal, Equals, retType.Decimal) - c.Assert(cType.Flag, Equals, retType.Flag) - c.Assert(lConvert, Equals, lCon) - c.Assert(rConvert, Equals, rCon) + require.Equal(t, retType.Tp, cType.Tp) + require.Equal(t, retType.Flen, cType.Flen) + require.Equal(t, retType.Decimal, cType.Decimal) + require.Equal(t, retType.Flag, cType.Flag) + require.Equal(t, lConvert, lCon) + require.Equal(t, rConvert, rCon) + } -func (t *testPlannerFunctionSuite) TestMPPJoinKeyTypeConvert(c *C) { +func TestMPPJoinKeyTypeConvert(t *testing.T) { tinyIntType := &types.FieldType{ Tp: mysql.TypeTiny, } @@ -100,13 +98,13 @@ func (t *testPlannerFunctionSuite) TestMPPJoinKeyTypeConvert(c *C) { Decimal: 0, } - testJoinKeyTypeConvert(tinyIntType, tinyIntType, tinyIntType, false, false, c) - testJoinKeyTypeConvert(tinyIntType, unsignedTinyIntType, bigIntType, true, true, c) - testJoinKeyTypeConvert(tinyIntType, bigIntType, bigIntType, true, false, c) - testJoinKeyTypeConvert(bigIntType, tinyIntType, bigIntType, false, true, c) - testJoinKeyTypeConvert(unsignedBigIntType, tinyIntType, decimalType, true, true, c) - testJoinKeyTypeConvert(tinyIntType, unsignedBigIntType, decimalType, true, true, c) - testJoinKeyTypeConvert(bigIntType, bigIntType, bigIntType, false, false, c) - testJoinKeyTypeConvert(unsignedBigIntType, bigIntType, decimalType, true, true, c) - testJoinKeyTypeConvert(bigIntType, unsignedBigIntType, decimalType, true, true, c) + testJoinKeyTypeConvert(t, tinyIntType, tinyIntType, tinyIntType, false, false) + testJoinKeyTypeConvert(t, tinyIntType, unsignedTinyIntType, bigIntType, true, true) + testJoinKeyTypeConvert(t, tinyIntType, bigIntType, bigIntType, true, false) + testJoinKeyTypeConvert(t, bigIntType, tinyIntType, bigIntType, false, true) + testJoinKeyTypeConvert(t, unsignedBigIntType, tinyIntType, decimalType, true, true) + testJoinKeyTypeConvert(t, tinyIntType, unsignedBigIntType, decimalType, true, true) + testJoinKeyTypeConvert(t, bigIntType, bigIntType, bigIntType, false, false) + testJoinKeyTypeConvert(t, unsignedBigIntType, bigIntType, decimalType, true, true) + testJoinKeyTypeConvert(t, bigIntType, unsignedBigIntType, decimalType, true, true) } diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index 9af2d94e8e8d3..6ecafac2532a6 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -19,47 +19,23 @@ import ( "fmt" "sort" "strings" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/sessionctx" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testPartitionPruneSuit{}) - -type testPartitionPruneSuit struct { - store kv.Storage - dom *domain.Domain - ctx sessionctx.Context - testData testutil.TestData -} - -func (s *testPartitionPruneSuit) SetUpSuite(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) - s.ctx = mock.NewContext() - s.testData, err = testutil.LoadTestSuiteData("testdata", "partition_pruner") - c.Assert(err, IsNil) -} - -func (s *testPartitionPruneSuit) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) - s.dom.Close() - s.store.Close() -} - -func (s *testPartitionPruneSuit) TestHashPartitionPruner(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashPartitionPruner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("create database test_partition") tk.MustExec("use test_partition") tk.MustExec("drop table if exists t1, t2;") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table t2(id int, a int, b int, primary key(id, a)) partition by hash(id + a) partitions 10;") tk.MustExec("create table t1(id int primary key, a int, b int) partition by hash(id) partitions 10;") tk.MustExec("create table t3(id int, a int, b int, primary key(id, a)) partition by hash(id) partitions 10;") @@ -76,18 +52,21 @@ func (s *testPartitionPruneSuit) TestHashPartitionPruner(c *C) { SQL string Result []string } - s.testData.GetTestCases(c, &input, &output) + partitionPrunerData := plannercore.GetPartitionPrunerData() + partitionPrunerData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } -func (s *testPartitionPruneSuit) TestRangeColumnPartitionPruningForIn(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangeColumnPartitionPruningForIn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_range_col_in") tk.MustExec("create database test_range_col_in") tk.MustExec("use test_range_col_in") @@ -163,12 +142,14 @@ func (s *testPartitionPruneSuit) TestRangeColumnPartitionPruningForIn(c *C) { " └─TableFullScan 10000.00 cop[tikv] table:t3, partition:p2 keep order:false, stats:pseudo")) } -func (s *testPartitionPruneSuit) TestListPartitionPruner(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestListPartitionPruner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("drop database if exists test_partition;") tk.MustExec("create database test_partition") tk.MustExec("use test_partition") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk.MustExec("create table t1 (id int, a int, b int ) partition by list ( a ) (partition p0 values in (1,2,3,4,5), partition p1 values in (6,7,8,9,10,null));") @@ -187,7 +168,7 @@ func (s *testPartitionPruneSuit) TestListPartitionPruner(c *C) { tk.MustExec("insert into t7 values (null),(0),(1),(2);") // tk2 use to compare the result with normal table. - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("drop database if exists test_partition_2;") tk2.MustExec("create database test_partition_2") tk2.MustExec("use test_partition_2") @@ -212,13 +193,14 @@ func (s *testPartitionPruneSuit) TestListPartitionPruner(c *C) { Result []string Plan []string } - s.testData.GetTestCases(c, &input, &output) + partitionPrunerData := plannercore.GetPartitionPrunerData() + partitionPrunerData.GetTestCases(t, &input, &output) valid := false for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) result := tk.MustQuery(tt) @@ -228,12 +210,14 @@ func (s *testPartitionPruneSuit) TestListPartitionPruner(c *C) { result.Check(tk2.MustQuery(tt).Rows()) valid = true } - c.Assert(valid, IsTrue) + require.True(t, valid) } } -func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestListColumnsPartitionPruner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("drop database if exists test_partition;") tk.MustExec("create database test_partition") @@ -245,7 +229,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { tk.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") // tk1 use to test partition table with index. - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("drop database if exists test_partition_1;") tk1.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk1.MustExec("create database test_partition_1") @@ -257,7 +241,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { tk1.MustExec("insert into t2 (id,a,b) values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6),(7,7,7),(8,8,8),(9,9,9),(10,10,10),(null,null,null)") // tk2 use to compare the result with normal table. - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("drop database if exists test_partition_2;") tk2.MustExec(`set @@session.tidb_regard_null_as_point=false`) tk2.MustExec("create database test_partition_2") @@ -277,18 +261,19 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { Plan []string IndexPlan []string } - s.testData.GetTestCases(c, &input, &output) + partitionPrunerData := plannercore.GetPartitionPrunerData() + partitionPrunerData.GetTestCases(t, &input, &output) valid := false for i, tt := range input { // Test for table without index. plan := tk.MustQuery("explain format = 'brief' " + tt.SQL) - planTree := s.testData.ConvertRowsToStrings(plan.Rows()) + planTree := testdata.ConvertRowsToStrings(plan.Rows()) // Test for table with index. indexPlan := tk1.MustQuery("explain format = 'brief' " + tt.SQL) - indexPlanTree := s.testData.ConvertRowsToStrings(indexPlan.Rows()) - s.testData.OnRecord(func() { + indexPlanTree := testdata.ConvertRowsToStrings(indexPlan.Rows()) + testdata.OnRecord(func() { output[i].SQL = tt.SQL - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt.SQL).Rows()) // Test for table without index. output[i].Plan = planTree // Test for table with index. @@ -299,8 +284,8 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { indexPlan.Check(testkit.Rows(output[i].IndexPlan...)) // compare the pruner information. - s.checkPrunePartitionInfo(c, tt.SQL, tt.Pruner, planTree) - s.checkPrunePartitionInfo(c, tt.SQL, tt.Pruner, indexPlanTree) + checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, planTree) + checkPrunePartitionInfo(t, tt.SQL, tt.Pruner, indexPlanTree) // compare the result. result := tk.MustQuery(tt.SQL) @@ -314,12 +299,13 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPruner(c *C) { valid = true } } - c.Assert(valid, IsTrue) + require.True(t, valid) } -func (s *testPartitionPruneSuit) checkPrunePartitionInfo(c *C, query string, infos1 string, plan []string) { - infos2 := s.getPartitionInfoFromPlan(plan) - c.Assert(infos1, Equals, infos2, Commentf("the query is: %v, the plan is:\n%v", query, strings.Join(plan, "\n"))) +func checkPrunePartitionInfo(c *testing.T, query string, infos1 string, plan []string) { + infos2 := getPartitionInfoFromPlan(plan) + comment := fmt.Sprintf("the query is: %v, the plan is:\n%v", query, strings.Join(plan, "\n")) + require.Equal(c, infos1, infos2, comment) } type testTablePartitionInfo struct { @@ -338,16 +324,16 @@ type testTablePartitionInfo struct { // " └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" // // The return table partition info is: t1: p0; t2: p1 -func (s *testPartitionPruneSuit) getPartitionInfoFromPlan(plan []string) string { +func getPartitionInfoFromPlan(plan []string) string { infos := make([]testTablePartitionInfo, 0, 2) info := testTablePartitionInfo{} for _, row := range plan { - partitions := s.getFieldValue("partition:", row) + partitions := getFieldValue("partition:", row) if partitions != "" { info.Partitions = partitions continue } - tbl := s.getFieldValue("table:", row) + tbl := getFieldValue("table:", row) if tbl != "" { info.Table = tbl infos = append(infos, info) @@ -369,7 +355,7 @@ func (s *testPartitionPruneSuit) getPartitionInfoFromPlan(plan []string) string return buf.String() } -func (s *testPartitionPruneSuit) getFieldValue(prefix, row string) string { +func getFieldValue(prefix, row string) string { if idx := strings.Index(row, prefix); idx > 0 { start := idx + len(prefix) end := strings.Index(row[start:], " ") @@ -382,8 +368,10 @@ func (s *testPartitionPruneSuit) getFieldValue(prefix, row string) string { return "" } -func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestListColumnsPartitionPrunerRandom(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) valueNum := 10 // Create table. tk.MustExec("drop database if exists test_partition;") @@ -392,7 +380,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk.MustExec("set @@session.tidb_enable_list_partition = ON") tk.MustExec("create table t1 (id int, a int, b int ) partition by list columns (b, id, a) (partition p0 values in ((1,0,2),(2,0,2),(0,1,0),(1,1,0),(2,1,0),(0,1,1),(0,1,2),(0,2,0),(1,2,0)),partition p1 values in ((1,0,1),(0,0,2),(2,1,1),(2,1,2),(2,2,1),(1,2,2),(2,2,2)),partition p2 values in ((0,0,0),(1,0,0),(2,0,0),(0,0,1),(2,0,1),(1,1,1),(1,1,2),(2,2,0),(0,2,1),(1,2,1),(0,2,2)))") - tk1 := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("drop database if exists test_partition_1;") tk1.MustExec("create database test_partition_1") tk1.MustExec("use test_partition_1") @@ -435,8 +423,10 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { } } -func (s *testPartitionPruneSuit) TestIssue22635(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22635(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS t1") tk.MustExec(` @@ -461,8 +451,10 @@ PARTITIONS 4`) tk.MustQuery("SELECT (SELECT tt.a FROM t1 tt LIMIT 1) aa, COUNT(DISTINCT b) FROM t1 GROUP BY aa").Check(testkit.Rows("4 4")) } -func (s *testPartitionPruneSuit) TestIssue22898(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue22898(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS test;") tk.MustExec("CREATE TABLE NT_RP3763 (COL1 TINYINT(8) SIGNED COMMENT \"NUMERIC NO INDEX\" DEFAULT 41,COL2 VARCHAR(20),COL3 DATETIME,COL4 BIGINT,COL5 FLOAT) PARTITION BY RANGE (COL1 * COL3) (PARTITION P0 VALUES LESS THAN (0),PARTITION P1 VALUES LESS THAN (10),PARTITION P2 VALUES LESS THAN (20),PARTITION P3 VALUES LESS THAN (30),PARTITION P4 VALUES LESS THAN (40),PARTITION P5 VALUES LESS THAN (50),PARTITION PMX VALUES LESS THAN MAXVALUE);") @@ -472,8 +464,10 @@ func (s *testPartitionPruneSuit) TestIssue22898(c *C) { tk.MustQuery("select * from `NT_RP3763` where `COL1` in (48);").Check(testkit.Rows("48 簖鹩筈匹眜赖泽騈爷詵赺玡婙Ɇ郝鮙廛賙疼舢 7228-12-13 02:59:54 -6181009269190017937 277311060000000000000000000000000000000")) } -func (s *testPartitionPruneSuit) TestIssue23622(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23622(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") tk.MustExec("drop table if exists t2;") tk.MustExec("create table t2 (a int, b int) partition by range (a) (partition p0 values less than (0), partition p1 values less than (5));") @@ -481,8 +475,10 @@ func (s *testPartitionPruneSuit) TestIssue23622(c *C) { tk.MustQuery("select * from t2 where a > 10 or b is NULL order by a;").Check(testkit.Rows("-1 ", "1 ")) } -func (s *testPartitionPruneSuit) Test22396(c *C) { - tk := testkit.NewTestKit(c, s.store) +func Test22396(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS test;") tk.MustExec("CREATE TABLE test(a INT, b INT, PRIMARY KEY(a, b)) PARTITION BY RANGE (a + b) (PARTITION p0 VALUES LESS THAN (20),PARTITION p1 VALUES LESS THAN MAXVALUE);") @@ -493,8 +489,10 @@ func (s *testPartitionPruneSuit) Test22396(c *C) { tk.MustQuery("SELECT * FROM test WHERE a + b = 2;") } -func (s *testPartitionPruneSuit) TestIssue23608(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23608(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("drop table if exists t1") @@ -539,14 +537,16 @@ partition by range (a) ( } //issue 22079 -func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestRangePartitionPredicatePruner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") tk.MustExec("drop database if exists test_partition;") tk.MustExec("create database test_partition") tk.MustExec("use test_partition") tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin partition by range(a) ( partition p0 values less than (1), @@ -559,18 +559,21 @@ func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { SQL string Result []string } - s.testData.GetTestCases(c, &input, &output) + partitionPrunerData := plannercore.GetPartitionPrunerData() + partitionPrunerData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) }) tk.MustQuery(tt).Check(testkit.Rows(output[i].Result...)) } } -func (s *testPartitionPruneSuit) TestHashPartitionPruning(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestHashPartitionPruning(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS t;") diff --git a/planner/core/partition_pruning_test.go b/planner/core/partition_pruning_test.go index 4ba103b01ea96..2b7e8625b15f5 100644 --- a/planner/core/partition_pruning_test.go +++ b/planner/core/partition_pruning_test.go @@ -15,7 +15,8 @@ package core import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser" @@ -23,17 +24,11 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testPartitionPruningSuite{}) - -type testPartitionPruningSuite struct { - partitionProcessor -} - -func (s *testPartitionPruningSuite) TestCanBePrune(c *C) { +func TestCanBePrune(t *testing.T) { // For the following case: // CREATE TABLE t1 ( recdate DATETIME NOT NULL ) // PARTITION BY RANGE( TO_DAYS(recdate) ) ( @@ -43,20 +38,17 @@ func (s *testPartitionPruningSuite) TestCanBePrune(c *C) { // SELECT * FROM t1 WHERE recdate < '2007-03-08 00:00:00'; // SELECT * FROM t1 WHERE recdate > '2018-03-08 00:00:00'; - tc := prepareTestCtx(c, - "create table t (d datetime not null)", - "to_days(d)", - ) + tc := prepareTestCtx(t, "create table t (d datetime not null)", "to_days(d)") lessThan := lessThanDataInt{data: []int64{733108, 733132}, maxvalue: false} - prunner := &rangePruner{lessThan, tc.col, tc.fn, monotoneModeNonStrict} + pruner := &rangePruner{lessThan, tc.col, tc.fn, monotoneModeNonStrict} queryExpr := tc.expr("d < '2000-03-08 00:00:00'") - result := partitionRangeForCNFExpr(tc.sctx, queryExpr, prunner, fullRange(len(lessThan.data))) - c.Assert(equalPartitionRangeOR(result, partitionRangeOR{{0, 1}}), IsTrue) + result := partitionRangeForCNFExpr(tc.sctx, queryExpr, pruner, fullRange(len(lessThan.data))) + require.True(t, equalPartitionRangeOR(result, partitionRangeOR{{0, 1}})) queryExpr = tc.expr("d > '2018-03-08 00:00:00'") - result = partitionRangeForCNFExpr(tc.sctx, queryExpr, prunner, fullRange(len(lessThan.data))) - c.Assert(equalPartitionRangeOR(result, partitionRangeOR{}), IsTrue) + result = partitionRangeForCNFExpr(tc.sctx, queryExpr, pruner, fullRange(len(lessThan.data))) + require.True(t, equalPartitionRangeOR(result, partitionRangeOR{})) // For the following case: // CREATE TABLE quarterly_report_status ( @@ -69,27 +61,24 @@ func (s *testPartitionPruningSuite) TestCanBePrune(c *C) { // PARTITION p2 VALUES LESS THAN (UNIX_TIMESTAMP('2010-01-01 00:00:00')), // PARTITION p3 VALUES LESS THAN (MAXVALUE) // ); - tc = prepareTestCtx(c, - "create table t (report_updated timestamp)", - "unix_timestamp(report_updated)", - ) + tc = prepareTestCtx(t, "create table t (report_updated timestamp)", "unix_timestamp(report_updated)") lessThan = lessThanDataInt{data: []int64{1199145600, 1207008000, 1262304000, 0}, maxvalue: true} - prunner = &rangePruner{lessThan, tc.col, tc.fn, monotoneModeStrict} + pruner = &rangePruner{lessThan, tc.col, tc.fn, monotoneModeStrict} queryExpr = tc.expr("report_updated > '2008-05-01 00:00:00'") - result = partitionRangeForCNFExpr(tc.sctx, queryExpr, prunner, fullRange(len(lessThan.data))) - c.Assert(equalPartitionRangeOR(result, partitionRangeOR{{2, 4}}), IsTrue) + result = partitionRangeForCNFExpr(tc.sctx, queryExpr, pruner, fullRange(len(lessThan.data))) + require.True(t, equalPartitionRangeOR(result, partitionRangeOR{{2, 4}})) queryExpr = tc.expr("report_updated > unix_timestamp('2008-05-01 00:00:00')") - partitionRangeForCNFExpr(tc.sctx, queryExpr, prunner, fullRange(len(lessThan.data))) + partitionRangeForCNFExpr(tc.sctx, queryExpr, pruner, fullRange(len(lessThan.data))) // TODO: Uncomment the check after fixing issue https://github.com/pingcap/tidb/issues/12028 - // c.Assert(equalPartitionRangeOR(result, partitionRangeOR{{2, 4}}), IsTrue) + // require.True(t, equalPartitionRangeOR(result, partitionRangeOR{{2, 4}})) // report_updated > unix_timestamp('2008-05-01 00:00:00') is converted to gt(t.t.report_updated, ) // Because unix_timestamp('2008-05-01 00:00:00') is fold to constant int 1564761600, and compare it with timestamp (report_updated) // need to convert 1564761600 to a timestamp, during that step, an error happen and the result is set to } -func (s *testPartitionPruningSuite) TestPruneUseBinarySearch(c *C) { +func TestPruneUseBinarySearch(t *testing.T) { lessThan := lessThanDataInt{data: []int64{4, 7, 11, 14, 17, 0}, maxvalue: true} cases := []struct { input dataForPrune @@ -122,13 +111,13 @@ func (s *testPartitionPruningSuite) TestPruneUseBinarySearch(c *C) { for i, ca := range cases { start, end := pruneUseBinarySearch(lessThan, ca.input, false) - c.Assert(ca.result.start, Equals, start, Commentf("fail = %d", i)) - c.Assert(ca.result.end, Equals, end, Commentf("fail = %d", i)) + require.Equalf(t, ca.result.start, start, "fail = %d", i) + require.Equalf(t, ca.result.end, end, "fail = %d", i) } } type testCtx struct { - c *C + require *require.Assertions sctx sessionctx.Context schema *expression.Schema columns []*expression.Column @@ -137,21 +126,21 @@ type testCtx struct { fn *expression.ScalarFunction } -func prepareTestCtx(c *C, createTable string, partitionExpr string) *testCtx { +func prepareTestCtx(t *testing.T, createTable string, partitionExpr string) *testCtx { p := parser.New() stmt, err := p.ParseOneStmt(createTable, "", "") - c.Assert(err, IsNil) + require.NoError(t, err) sctx := mock.NewContext() tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) - c.Assert(err, IsNil) + require.NoError(t, err) columns, names, err := expression.ColumnInfos2ColumnsAndNames(sctx, model.NewCIStr("t"), tblInfo.Name, tblInfo.Cols(), tblInfo) - c.Assert(err, IsNil) + require.NoError(t, err) schema := expression.NewSchema(columns...) col, fn, _, err := makePartitionByFnCol(sctx, columns, names, partitionExpr) - c.Assert(err, IsNil) + require.NoError(t, err) return &testCtx{ - c: c, + require: require.New(t), sctx: sctx, schema: schema, columns: columns, @@ -163,15 +152,12 @@ func prepareTestCtx(c *C, createTable string, partitionExpr string) *testCtx { func (tc *testCtx) expr(expr string) []expression.Expression { res, err := expression.ParseSimpleExprsWithNames(tc.sctx, expr, tc.schema, tc.names) - tc.c.Assert(err, IsNil) + tc.require.NoError(err) return res } -func (s *testPartitionPruningSuite) TestPartitionRangeForExpr(c *C) { - tc := prepareTestCtx(c, - "create table t (a int)", - "a", - ) +func TestPartitionRangeForExpr(t *testing.T) { + tc := prepareTestCtx(t, "create table t (a int)", "a") lessThan := lessThanDataInt{data: []int64{4, 7, 11, 14, 17, 0}, maxvalue: true} prunner := &rangePruner{lessThan, tc.columns[0], nil, monotoneModeInvalid} cases := []struct { @@ -194,10 +180,10 @@ func (s *testPartitionPruningSuite) TestPartitionRangeForExpr(c *C) { for _, ca := range cases { expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) result := fullRange(lessThan.length()) result = partitionRangeForExpr(tc.sctx, expr[0], prunner, result) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("unexpected:", ca.input)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "unexpected: %v", ca.input) } } @@ -213,7 +199,7 @@ func equalPartitionRangeOR(x, y partitionRangeOR) bool { return true } -func (s *testPartitionPruningSuite) TestPartitionRangeOperation(c *C) { +func TestPartitionRangeOperation(t *testing.T) { testIntersectionRange := []struct { input1 partitionRangeOR input2 partitionRange @@ -231,7 +217,7 @@ func (s *testPartitionPruningSuite) TestPartitionRangeOperation(c *C) { } for i, ca := range testIntersectionRange { result := ca.input1.intersectionRange(ca.input2.start, ca.input2.end) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("failed %d", i)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "fail = %d", i) } testIntersection := []struct { @@ -251,7 +237,7 @@ func (s *testPartitionPruningSuite) TestPartitionRangeOperation(c *C) { } for i, ca := range testIntersection { result := ca.input1.intersection(ca.input2) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("failed %d", i)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "fail = %d", i) } testUnion := []struct { @@ -271,24 +257,21 @@ func (s *testPartitionPruningSuite) TestPartitionRangeOperation(c *C) { } for i, ca := range testUnion { result := ca.input1.union(ca.input2) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("failed %d", i)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "fail = %d", i) } } -func (s *testPartitionPruningSuite) TestPartitionRangePrunner2VarChar(c *C) { - tc := prepareTestCtx(c, - "create table t (a varchar(32))", - "a", - ) +func TestPartitionRangePrunner2VarChar(t *testing.T) { + tc := prepareTestCtx(t, "create table t (a varchar(32))", "a") lessThanDataInt := []string{"'c'", "'f'", "'h'", "'l'", "'t'"} lessThan := make([]expression.Expression, len(lessThanDataInt)+1) // +1 for maxvalue for i, str := range lessThanDataInt { tmp, err := expression.ParseSimpleExprsWithNames(tc.sctx, str, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) lessThan[i] = tmp[0] } - prunner := &rangeColumnsPruner{lessThan, tc.columns[0], true} + pruner := &rangeColumnsPruner{lessThan, tc.columns[0], true} cases := []struct { input string result partitionRangeOR @@ -310,17 +293,15 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2VarChar(c *C) { for _, ca := range cases { expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) result := fullRange(len(lessThan)) - result = partitionRangeForExpr(tc.sctx, expr[0], prunner, result) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("unexpected:", ca.input)) + result = partitionRangeForExpr(tc.sctx, expr[0], pruner, result) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "unexpected: %v", ca.input) } } -func (s *testPartitionPruningSuite) TestPartitionRangePrunner2CharWithCollation(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - tc := prepareTestCtx(c, +func TestPartitionRangePrunner2CharWithCollation(t *testing.T) { + tc := prepareTestCtx(t, "create table t (a char(32) collate utf8mb4_unicode_ci)", "a", ) @@ -328,7 +309,7 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2CharWithCollation( lessThan := make([]expression.Expression, len(lessThanDataInt)+1) // +1 for maxvalue for i, str := range lessThanDataInt { tmp, err := expression.ParseSimpleExprsWithNames(tc.sctx, str, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) lessThan[i] = tmp[0] } @@ -356,15 +337,15 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2CharWithCollation( for _, ca := range cases { expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) result := fullRange(len(lessThan)) result = partitionRangeForExpr(tc.sctx, expr[0], prunner, result) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("unexpected:", ca.input)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "unexpected: %v", ca.input) } } -func (s *testPartitionPruningSuite) TestPartitionRangePrunner2Date(c *C) { - tc := prepareTestCtx(c, +func TestPartitionRangePrunner2Date(t *testing.T) { + tc := prepareTestCtx(t, "create table t (a date)", "a", ) @@ -378,7 +359,7 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2Date(c *C) { lessThan := make([]expression.Expression, len(lessThanDataInt)) for i, str := range lessThanDataInt { tmp, err := expression.ParseSimpleExprsWithNames(tc.sctx, str, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) lessThan[i] = tmp[0] } @@ -403,9 +384,9 @@ func (s *testPartitionPruningSuite) TestPartitionRangePrunner2Date(c *C) { for _, ca := range cases { expr, err := expression.ParseSimpleExprsWithNames(tc.sctx, ca.input, tc.schema, tc.names) - c.Assert(err, IsNil) + require.NoError(t, err) result := fullRange(len(lessThan)) result = partitionRangeForExpr(tc.sctx, expr[0], prunner, result) - c.Assert(equalPartitionRangeOR(ca.result, result), IsTrue, Commentf("unexpected:", ca.input)) + require.Truef(t, equalPartitionRangeOR(ca.result, result), "unexpected: %v", ca.input) } } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index f77ef93e20de1..21e810f7c1ccc 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -19,8 +19,8 @@ import ( "fmt" "math" "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" @@ -32,103 +32,51 @@ import ( "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/hint" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testPlanSuite{}) -var _ = SerialSuites(&testPlanSerialSuite{}) +func TestDAGPlanBuilderSimpleCase(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() -type testPlanSuiteBase struct { - *parser.Parser - is infoschema.InfoSchema -} - -func (s *testPlanSuiteBase) SetUpSuite(c *C) { - s.is = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) - s.Parser = parser.New() - s.Parser.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) -} - -type testPlanSerialSuite struct { - testPlanSuiteBase -} - -type testPlanSuite struct { - testPlanSuiteBase - - testData testutil.TestData -} - -func (s *testPlanSuite) SetUpSuite(c *C) { - s.testPlanSuiteBase.SetUpSuite(c) - - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "plan_suite") - c.Assert(err, IsNil) -} - -func (s *testPlanSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testPlanSuite) TestDAGPlanBuilderSimpleCase(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set tidb_opt_limit_push_down_threshold=0") - c.Assert(err, IsNil) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_opt_limit_push_down_threshold=0") var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - err = se.NewTxn(context.Background()) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case: %v, sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + require.NoError(t, tk.Session().NewTxn(context.Background())) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestAnalyzeBuildSucc(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - sctx := se.(sessionctx.Context) - _, err = se.Execute(context.Background(), "create table t(a int)") - c.Assert(err, IsNil) +func TestAnalyzeBuildSucc(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int)") tests := []struct { sql string succ bool @@ -160,43 +108,37 @@ func (s *testPlanSuite) TestAnalyzeBuildSucc(c *C) { statsVer: 1, }, } + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range tests { - comment := Commentf("The %v-th test failed", i) - _, err := se.Execute(context.Background(), fmt.Sprintf("set @@tidb_analyze_version=%v", tt.statsVer)) - c.Assert(err, IsNil) + comment := fmt.Sprintf("The %v-th test failed", i) + tk.MustExec(fmt.Sprintf("set @@tidb_analyze_version=%v", tt.statsVer)) - stmt, err := s.ParseOneStmt(tt.sql, "", "") + stmt, err := p.ParseOneStmt(tt.sql, "", "") if tt.succ { - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) } else if err != nil { continue } - err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) - c.Assert(err, IsNil) - _, _, err = planner.Optimize(context.Background(), sctx, stmt, s.is) + err = core.Preprocess(tk.Session(), stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + _, _, err = planner.Optimize(context.Background(), tk.Session(), stmt, is) if tt.succ { - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) } else { - c.Assert(err, NotNil, comment) + require.Error(t, err, comment) } } } -func (s *testPlanSuite) TestAnalyzeSetRate(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - sctx := se.(sessionctx.Context) - _, err = se.Execute(context.Background(), "create table t(a int)") - c.Assert(err, IsNil) +func TestAnalyzeSetRate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int)") tests := []struct { sql string rate float64 @@ -214,35 +156,30 @@ func (s *testPlanSuite) TestAnalyzeSetRate(c *C) { rate: -1, }, } + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range tests { - comment := Commentf("The %v-th test failed", i) - c.Assert(err, IsNil) - - stmt, err := s.ParseOneStmt(tt.sql, "", "") - c.Assert(err, IsNil, comment) - err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.Background(), sctx, stmt, s.is) - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("The %v-th test failed", i) + stmt, err := p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + + err = core.Preprocess(tk.Session(), stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err, comment) + p, _, err := planner.Optimize(context.Background(), tk.Session(), stmt, is) + require.NoError(t, err, comment) ana := p.(*core.Analyze) - c.Assert(math.Float64frombits(ana.Opts[ast.AnalyzeOptSampleRate]), Equals, tt.rate) + require.Equal(t, tt.rate, math.Float64frombits(ana.Opts[ast.AnalyzeOptSampleRate])) } } -func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - ctx := se.(sessionctx.Context) - sessionVars := ctx.GetSessionVars() +func TestDAGPlanBuilderJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + sessionVars := tk.Session().GetSessionVars() sessionVars.ExecutorConcurrency = 4 sessionVars.SetDistSQLScanConcurrency(15) sessionVars.SetHashJoinConcurrency(5) @@ -252,38 +189,34 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set sql_mode='STRICT_TRANS_TABLES'") - c.Assert(err, IsNil) // disable only full group by - ctx := se.(sessionctx.Context) - sessionVars := ctx.GetSessionVars() +func TestDAGPlanBuilderSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + sessionVars := tk.Session().GetSessionVars() sessionVars.SetHashAggFinalConcurrency(1) sessionVars.SetHashAggPartialConcurrency(1) sessionVars.SetHashJoinConcurrency(5) @@ -294,69 +227,64 @@ func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, Commentf("for %s", tt)) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) } } -func (s *testPlanSuite) TestDAGPlanTopN(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestDAGPlanTopN(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) +func TestDAGPlanBuilderBasePhysicalPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + se, err := session.CreateSession4Test(store) + require.NoError(t, err) _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) + require.NoError(t, err) var input []string var output []struct { @@ -364,117 +292,103 @@ func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { Best string Hints string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), se, stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Best, Commentf("for %s", tt)) - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, Commentf("for %s", tt)) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), fmt.Sprintf("input: %s", tt)) } } -func (s *testPlanSuite) TestDAGPlanBuilderUnion(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestDAGPlanBuilderUnion(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestDAGPlanBuilderUnionScan(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestDAGPlanBuilderUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Best string } + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + require.NoError(t, tk.Session().NewTxn(context.Background())) - err = se.NewTxn(context.Background()) - c.Assert(err, IsNil) // Make txn not read only. - txn, err := se.Txn(true) - c.Assert(err, IsNil) + txn, err := tk.Session().Txn(true) + require.NoError(t, err) err = txn.Set(kv.Key("AAA"), []byte("BBB")) - c.Assert(err, IsNil) - se.StmtCommit() - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + require.NoError(t, err) + tk.Session().StmtCommit() + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, Commentf("for %s", tt)) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) } } -func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set sql_mode='STRICT_TRANS_TABLES'") - c.Assert(err, IsNil) // disable only full group by - ctx := se.(sessionctx.Context) - sessionVars := ctx.GetSessionVars() +func TestDAGPlanBuilderAgg(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by + sessionVars := tk.Session().GetSessionVars() sessionVars.SetHashAggFinalConcurrency(1) sessionVars.SetHashAggPartialConcurrency(1) sessionVars.SetDistSQLScanConcurrency(15) @@ -485,96 +399,91 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, Commentf("for %s", tt)) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) } } -func (s *testPlanSuite) TestRefine(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestRefine(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - sc := se.(sessionctx.Context).GetSessionVars().StmtCtx + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + sc := tk.Session().GetSessionVars().StmtCtx sc.IgnoreTruncate = false - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestAggEliminator(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set tidb_opt_limit_push_down_threshold=0") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set sql_mode='STRICT_TRANS_TABLES'") - c.Assert(err, IsNil) // disable only full group by +func TestAggEliminator(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_opt_limit_push_down_threshold=0") + tk.MustExec("set sql_mode='STRICT_TRANS_TABLES'") // disable only full group by var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - comment := Commentf("for %s", tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - sc := se.(sessionctx.Context).GetSessionVars().StmtCtx + comment := fmt.Sprintf("input: %s", tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + sc := tk.Session().GetSessionVars().StmtCtx sc.IgnoreTruncate = false - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, Commentf("for %s", tt)) + require.Equal(t, output[i].Best, core.ToString(p), fmt.Sprintf("input: %s", tt)) } } -func (s *testPlanSuite) TestINMJHint(c *C) { +func TestINMJHint(t *testing.T) { var ( input []string output []struct { @@ -583,14 +492,11 @@ func (s *testPlanSuite) TestINMJHint(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int primary key, b int not null)") @@ -599,17 +505,17 @@ func (s *testPlanSuite) TestINMJHint(c *C) { tk.MustExec("insert into t2 values(1,1),(2,1)") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestEliminateMaxOneRow(c *C) { +func TestEliminateMaxOneRow(t *testing.T) { var ( input []string output []struct { @@ -618,14 +524,11 @@ func (s *testPlanSuite) TestEliminateMaxOneRow(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") tk.MustExec("drop table if exists t2;") @@ -635,10 +538,10 @@ func (s *testPlanSuite) TestEliminateMaxOneRow(c *C) { tk.MustExec("create table t3(a int(11) DEFAULT NULL, b int(11) DEFAULT NULL, c int(11) DEFAULT NULL, UNIQUE KEY idx_abc (a, b, c))") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) @@ -654,65 +557,57 @@ func (store overrideStore) GetClient() kv.Client { type overrideClient struct{ kv.Client } -func (cli overrideClient) IsRequestTypeSupported(reqType, subType int64) bool { +func (cli overrideClient) IsRequestTypeSupported(_, _ int64) bool { return false } -func (s *testPlanSuite) TestRequestTypeSupportedOff(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() +func TestRequestTypeSupportedOff(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() se, err := session.CreateSession4Test(overrideStore{store}) - c.Assert(err, IsNil) + require.NoError(t, err) _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) + require.NoError(t, err) sql := "select * from t where a in (1, 10, 20)" expect := "TableReader(Table(t))->Sel([in(test.t.a, 1, 10, 20)])" - stmt, err := s.ParseOneStmt(sql, "", "") - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - c.Assert(core.ToString(p), Equals, expect, Commentf("for %s", sql)) + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + stmt, err := parser.New().ParseOneStmt(sql, "", "") + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), se, stmt, is) + require.NoError(t, err) + require.Equal(t, expect, core.ToString(p), fmt.Sprintf("sql: %s", sql)) } -func (s *testPlanSuite) TestIndexJoinUnionScan(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIndexJoinUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("create table t (a int primary key, b int, index idx(a))") + tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") + tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) + var input [][]string var output []struct { SQL []string Plan []string } - tk.MustExec("create table t (a int primary key, b int, index idx(a))") - tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) - tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) - - s.testData.GetTestCases(c, &input, &output) for i, ts := range input { tk.MustExec("begin") for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -723,38 +618,34 @@ func (s *testPlanSuite) TestIndexJoinUnionScan(c *C) { } } -func (s *testPlanSuite) TestMergeJoinUnionScan(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - +func TestMergeJoinUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int))") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int))") + tk.MustExec("insert into t1 (`c_int`, `c_str`) values (11, 'keen williamson'), (10, 'gracious hermann')") + tk.MustExec("insert into t2 (`c_int`, `c_str`) values (10, 'gracious hermann')") + var input [][]string var output []struct { SQL []string Plan []string } - tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int))") - tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int))") - tk.MustExec("insert into t1 (`c_int`, `c_str`) values (11, 'keen williamson'), (10, 'gracious hermann')") - tk.MustExec("insert into t2 (`c_int`, `c_str`) values (10, 'gracious hermann')") + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) - s.testData.GetTestCases(c, &input, &output) for i, ts := range input { tk.MustExec("begin") for j, tt := range ts { if j != len(ts)-1 { tk.MustExec(tt) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts if j == len(ts)-1 { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) } }) if j == len(ts)-1 { @@ -765,18 +656,13 @@ func (s *testPlanSuite) TestMergeJoinUnionScan(c *C) { } } -func (s *testPlanSuite) TestDoSubquery(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestDoSubQuery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tests := []struct { sql string best string @@ -786,161 +672,145 @@ func (s *testPlanSuite) TestDoSubquery(c *C) { best: "LeftHashJoin{Dual->PointGet(Handle(t.a)1)}->Projection", }, } + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for _, tt := range tests { - comment := Commentf("for %s", tt.sql) - stmt, err := s.ParseOneStmt(tt.sql, "", "") - c.Assert(err, IsNil, comment) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - c.Assert(core.ToString(p), Equals, tt.best, comment) + comment := fmt.Sprintf("for %s", tt.sql) + stmt, err := p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + require.Equal(t, tt.best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestIndexLookupCartesianJoin(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - sql := "select /*+ TIDB_INLJ(t1, t2) */ * from t t1 join t t2" - stmt, err := s.ParseOneStmt(sql, "", "") - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - c.Assert(core.ToString(p), Equals, "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}") - warnings := se.GetSessionVars().StmtCtx.GetWarnings() +func TestIndexLookupCartesianJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + stmt, err := parser.New().ParseOneStmt("select /*+ TIDB_INLJ(t1, t2) */ * from t t1 join t t2", "", "") + require.NoError(t, err) + + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + require.Equal(t, "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}", core.ToString(p)) + + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() lastWarn := warnings[len(warnings)-1] err = core.ErrInternal.GenWithStack("TIDB_INLJ hint is inapplicable without column equal ON condition") - c.Assert(terror.ErrorEqual(err, lastWarn.Err), IsTrue) + require.True(t, terror.ErrorEqual(err, lastWarn.Err)) } -func (s *testPlanSuite) TestSemiJoinToInner(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestSemiJoinToInner(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, tt := range input { - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best) + require.Equal(t, output[i].Best, core.ToString(p)) } } -func (s *testPlanSuite) TestUnmatchedTableInHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestUnmatchedTableInHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Warning string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, test := range input { - se.GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil) - _, _, err = planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err) + _, _, err = planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { output[i].SQL = test if len(warnings) > 0 { output[i].Warning = warnings[0].Err.Error() } }) if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0) + require.Len(t, warnings, 0) } else { - c.Assert(len(warnings), Equals, 1) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning) + require.Len(t, warnings, 1) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) } } } -func (s *testPlanSuite) TestHintScope(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestHintScope(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql:%s", i, test) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(context.Background(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(context.Background(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = test output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best) - - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - c.Assert(warnings, HasLen, 0, comment) + require.Equal(t, output[i].Best, core.ToString(p)) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + require.Len(t, warnings, 0, comment) } } -func (s *testPlanSuite) TestJoinHints(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestJoinHints(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { @@ -949,19 +819,23 @@ func (s *testPlanSuite) TestJoinHints(c *C) { Warning string Hints string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql:%s", i, test) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = test output[i].Best = core.ToString(p) if len(warnings) > 0 { @@ -969,32 +843,26 @@ func (s *testPlanSuite) TestJoinHints(c *C) { } output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Best) + require.Equal(t, output[i].Best, core.ToString(p)) if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0) + require.Len(t, warnings, 0) } else { - c.Assert(len(warnings), Equals, 1, Commentf("%v", warnings)) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning) + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) } - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), comment) } } -func (s *testPlanSuite) TestAggregationHints(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestAggregationHints(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") - sessionVars := se.(sessionctx.Context).GetSessionVars() + sessionVars := tk.Session().GetSessionVars() sessionVars.SetHashAggFinalConcurrency(1) sessionVars.SetHashAggPartialConcurrency(1) @@ -1007,47 +875,45 @@ func (s *testPlanSuite) TestAggregationHints(c *C) { Best string Warning string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - se.GetSessionVars().AllowAggPushDown = test.AggPushDown + comment := fmt.Sprintf("case: %v sql: %v", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + tk.Session().GetSessionVars().AllowAggPushDown = test.AggPushDown - stmt, err := s.ParseOneStmt(test.SQL, "", "") - c.Assert(err, IsNil, comment) + stmt, err := p.ParseOneStmt(test.SQL, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = test.SQL output[i].Best = core.ToString(p) if len(warnings) > 0 { output[i].Warning = warnings[0].Err.Error() } }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0, comment) + require.Len(t, warnings, 0) } else { - c.Assert(len(warnings), Equals, 1, comment) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) } } } -func (s *testPlanSuite) TestExplainJoinHints(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestExplainJoinHints(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, key(b), key(c))") @@ -1059,15 +925,10 @@ func (s *testPlanSuite) TestExplainJoinHints(c *C) { )) } -func (s *testPlanSuite) TestAggToCopHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestAggToCopHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists ta") tk.MustExec("create table ta(a int, b int, index(a))") @@ -1080,55 +941,52 @@ func (s *testPlanSuite) TestAggToCopHint(c *C) { Warning string } ) - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.Background() - is := domain.GetDomain(tk.Se).InfoSchema() + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + testdata.OnRecord(func() { output[i].SQL = test }) - c.Assert(test, Equals, output[i].SQL, comment) + require.Equal(t, output[i].SQL, test, comment) - tk.Se.GetSessionVars().StmtCtx.SetWarnings(nil) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, tk.Se, stmt, is) - c.Assert(err, IsNil, comment) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) planString := core.ToString(p) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].Best = planString }) - c.Assert(planString, Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, planString, comment) - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { if len(warnings) > 0 { output[i].Warning = warnings[0].Err.Error() } }) if output[i].Warning == "" { - c.Assert(len(warnings), Equals, 0, comment) + require.Len(t, warnings, 0) } else { - c.Assert(len(warnings), Equals, 1, comment) - c.Assert(warnings[0].Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warnings[0].Err.Error(), Equals, output[i].Warning, comment) + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) } } } -func (s *testPlanSuite) TestLimitToCopHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestLimitToCopHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tn") tk.MustExec("create table tn(a int, b int, c int, d int, key (a, b, c, d))") @@ -1143,18 +1001,19 @@ func (s *testPlanSuite) TestLimitToCopHint(c *C) { } ) - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) - comment := Commentf("case:%v sql:%s", i, ts) - warnings := tk.Se.GetSessionVars().StmtCtx.GetWarnings() - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql:%s", i, ts) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + testdata.OnRecord(func() { if len(warnings) > 0 { output[i].Warning = make([]string, len(warnings)) for j, warning := range warnings { @@ -1163,19 +1022,18 @@ func (s *testPlanSuite) TestLimitToCopHint(c *C) { } }) if len(output[i].Warning) == 0 { - c.Assert(len(warnings), Equals, 0, comment) + require.Len(t, warnings, 0) } else { - c.Assert(len(warnings), Equals, len(output[i].Warning), comment) + require.Len(t, warnings, len(output[i].Warning), comment) for j, warning := range warnings { - c.Assert(warning.Level, Equals, stmtctx.WarnLevelWarning, comment) - c.Assert(warning.Err.Error(), Equals, output[i].Warning[j], comment) + require.Equal(t, stmtctx.WarnLevelWarning, warning.Level, comment) + require.Equal(t, output[i].Warning[j], warning.Err.Error(), comment) } } } } -func (s *testPlanSuite) TestPushdownDistinctEnable(c *C) { - defer testleak.AfterTest(c)() +func TestPushdownDistinctEnable(t *testing.T) { var ( input []string output []struct { @@ -1184,16 +1042,16 @@ func (s *testPlanSuite) TestPushdownDistinctEnable(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) vars := []string{ fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), "set session tidb_opt_agg_push_down = 1", } - s.doTestPushdownDistinct(c, vars, input, output) + doTestPushdownDistinct(t, vars, input, output) } -func (s *testPlanSuite) TestPushdownDistinctDisable(c *C) { - defer testleak.AfterTest(c)() +func TestPushdownDistinctDisable(t *testing.T) { var ( input []string output []struct { @@ -1203,15 +1061,16 @@ func (s *testPlanSuite) TestPushdownDistinctDisable(c *C) { } ) - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) vars := []string{ fmt.Sprintf("set @@session.%s = 0", variable.TiDBOptDistinctAggPushDown), "set session tidb_opt_agg_push_down = 1", } - s.doTestPushdownDistinct(c, vars, input, output) + doTestPushdownDistinct(t, vars, input, output) } -func (s *testPlanSuite) TestPushdownDistinctEnableAggPushDownDisable(c *C) { +func TestPushdownDistinctEnableAggPushDownDisable(t *testing.T) { var ( input []string output []struct { @@ -1220,26 +1079,23 @@ func (s *testPlanSuite) TestPushdownDistinctEnableAggPushDownDisable(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) vars := []string{ fmt.Sprintf("set @@session.%s = 1", variable.TiDBOptDistinctAggPushDown), "set session tidb_opt_agg_push_down = 0", } - s.doTestPushdownDistinct(c, vars, input, output) + doTestPushdownDistinct(t, vars, input, output) } -func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, output []struct { +func doTestPushdownDistinct(t *testing.T, vars, input []string, output []struct { SQL string Plan []string Result []string }) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1273,17 +1129,17 @@ func (s *testPlanSuite) doTestPushdownDistinct(c *C, vars, input []string, outpu } for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { +func TestGroupConcatOrderby(t *testing.T) { var ( input []string output []struct { @@ -1292,14 +1148,11 @@ func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists test;") tk.MustExec("create table test(id int, name int)") @@ -1318,28 +1171,22 @@ func (s *testPlanSuite) TestGroupConcatOrderby(c *C) { tk.MustExec(fmt.Sprintf("set session tidb_opt_agg_push_down = %v", 1)) for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestHintAlias(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestHintAlias(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tests := []struct { sql1 string @@ -1359,34 +1206,31 @@ func (s *testPlanSuite) TestHintAlias(c *C) { }, } ctx := context.TODO() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range tests { - comment := Commentf("case:%v sql1:%s sql2:%s", i, tt.sql1, tt.sql2) - stmt1, err := s.ParseOneStmt(tt.sql1, "", "") - c.Assert(err, IsNil, comment) - stmt2, err := s.ParseOneStmt(tt.sql2, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql1:%s sql2:%s", i, tt.sql1, tt.sql2) + stmt1, err := p.ParseOneStmt(tt.sql1, "", "") + require.NoError(t, err, comment) + stmt2, err := p.ParseOneStmt(tt.sql2, "", "") + require.NoError(t, err, comment) - p1, _, err := planner.Optimize(ctx, se, stmt1, s.is) - c.Assert(err, IsNil) - p2, _, err := planner.Optimize(ctx, se, stmt2, s.is) - c.Assert(err, IsNil) + p1, _, err := planner.Optimize(ctx, tk.Session(), stmt1, is) + require.NoError(t, err) + p2, _, err := planner.Optimize(ctx, tk.Session(), stmt2, is) + require.NoError(t, err) - c.Assert(core.ToString(p1), Equals, core.ToString(p2)) + require.Equal(t, core.ToString(p2), core.ToString(p1)) } } -func (s *testPlanSuite) TestIndexHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestIndexHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { @@ -1395,46 +1239,44 @@ func (s *testPlanSuite) TestIndexHint(c *C) { HasWarn bool Hints string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) + comment := fmt.Sprintf("case:%v sql:%s", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = test output[i].Best = core.ToString(p) - output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() + require.Equal(t, output[i].Best, core.ToString(p), comment) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() if output[i].HasWarn { - c.Assert(warnings, HasLen, 1, comment) + require.Len(t, warnings, 1, comment) } else { - c.Assert(warnings, HasLen, 0, comment) + require.Len(t, warnings, 0, comment) } - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), comment) } } -func (s *testPlanSuite) TestIndexMergeHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestIndexMergeHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { @@ -1443,47 +1285,45 @@ func (s *testPlanSuite) TestIndexMergeHint(c *C) { HasWarn bool Hints string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { - comment := Commentf("case:%v sql:%s", i, test) - se.GetSessionVars().StmtCtx.SetWarnings(nil) - stmt, err := s.ParseOneStmt(test, "", "") - c.Assert(err, IsNil, comment) - sctx := se.(sessionctx.Context) + comment := fmt.Sprintf("case:%v sql:%s", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + sctx := tk.Session() err = executor.ResetContextOfStmt(sctx, stmt) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + require.NoError(t, err) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = test output[i].Best = core.ToString(p) - output[i].HasWarn = len(se.GetSessionVars().StmtCtx.GetWarnings()) > 0 + output[i].HasWarn = len(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) > 0 output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) - warnings := se.GetSessionVars().StmtCtx.GetWarnings() + require.Equal(t, output[i].Best, core.ToString(p), comment) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() if output[i].HasWarn { - c.Assert(warnings, HasLen, 1, comment) + require.Len(t, warnings, 1, comment) } else { - c.Assert(warnings, HasLen, 0, comment) + require.Len(t, warnings, 0, comment) } - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), comment) } } -func (s *testPlanSuite) TestQueryBlockHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestQueryBlockHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") var input []string var output []struct { @@ -1491,44 +1331,38 @@ func (s *testPlanSuite) TestQueryBlockHint(c *C) { Plan string Hints string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) ctx := context.TODO() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, se, stmt, s.is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = core.ToString(p) output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), comment) } } -func (s *testPlanSuite) TestInlineProjection(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2;`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) +func TestInlineProjection(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.t1, test.t2;`) + tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) var input []string var output []struct { @@ -1536,37 +1370,36 @@ func (s *testPlanSuite) TestInlineProjection(c *C) { Plan string Hints string } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) + is := domain.GetDomain(tk.Session()).InfoSchema() + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + + ctx := context.Background() + p := parser.New() + for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = core.ToString(p) output[i].Hints = hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)) }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) - c.Assert(hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), Equals, output[i].Hints, comment) + require.Equal(t, output[i].Plan, core.ToString(p), comment) + require.Equal(t, output[i].Hints, hint.RestoreOptimizerHints(core.GenHintsFromPhysicalPlan(p)), comment) } } -func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) +func TestDAGPlanBuilderSplitAvg(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tests := []struct { sql string plan string @@ -1581,37 +1414,40 @@ func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { }, } + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for _, tt := range tests { - comment := Commentf("for %s", tt.sql) - stmt, err := s.ParseOneStmt(tt.sql, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("for %s", tt.sql) + stmt, err := p.ParseOneStmt(tt.sql, "", "") + require.NoError(t, err, comment) - err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil, comment) + err = core.Preprocess(tk.Session(), stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is})) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err, comment) - c.Assert(core.ToString(p), Equals, tt.plan, comment) + require.Equal(t, tt.plan, core.ToString(p), comment) root, ok := p.(core.PhysicalPlan) if !ok { continue } - testDAGPlanBuilderSplitAvg(c, root) + testDAGPlanBuilderSplitAvg(t, root) } } -func testDAGPlanBuilderSplitAvg(c *C, root core.PhysicalPlan) { +func testDAGPlanBuilderSplitAvg(t *testing.T, root core.PhysicalPlan) { if p, ok := root.(*core.PhysicalTableReader); ok { if p.TablePlans != nil { baseAgg := p.TablePlans[len(p.TablePlans)-1] if agg, ok := baseAgg.(*core.PhysicalHashAgg); ok { for i, aggfunc := range agg.AggFuncs { - c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) + require.Equal(t, aggfunc.RetTp, agg.Schema().Columns[i].RetType) } } if agg, ok := baseAgg.(*core.PhysicalStreamAgg); ok { for i, aggfunc := range agg.AggFuncs { - c.Assert(agg.Schema().Columns[i].RetType, Equals, aggfunc.RetTp) + require.Equal(t, aggfunc.RetTp, agg.Schema().Columns[i].RetType) } } } @@ -1622,141 +1458,122 @@ func testDAGPlanBuilderSplitAvg(c *C, root core.PhysicalPlan) { return } for _, son := range childs { - testDAGPlanBuilderSplitAvg(c, son) + testDAGPlanBuilderSplitAvg(t, son) } } -func (s *testPlanSuite) TestIndexJoinHint(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1, test.t2, test.t;`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") - c.Assert(err, IsNil) +func TestIndexJoinHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.t1, test.t2, test.t;`) + tk.MustExec(`create table test.t1(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec(`create table test.t2(a bigint, b bigint, index idx_a(a), index idx_b(b));`) + tk.MustExec("CREATE TABLE `t` ( `a` bigint(20) NOT NULL, `b` tinyint(1) DEFAULT NULL, `c` datetime DEFAULT NULL, `d` int(10) unsigned DEFAULT NULL, `e` varchar(20) DEFAULT NULL, `f` double DEFAULT NULL, `g` decimal(30,5) DEFAULT NULL, `h` float DEFAULT NULL, `i` date DEFAULT NULL, `j` timestamp NULL DEFAULT NULL, PRIMARY KEY (`a`), UNIQUE KEY `b` (`b`), KEY `c` (`c`,`d`,`e`), KEY `f` (`f`), KEY `g` (`g`,`h`), KEY `g_2` (`g`), UNIQUE KEY `g_3` (`g`), KEY `i` (`i`) );") + var input []string var output []struct { SQL string Plan string } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) + + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() + ctx := context.Background() + + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + require.Equal(t, output[i].Plan, core.ToString(p), comment) } } -func (s *testPlanSuite) TestDAGPlanBuilderWindow(c *C) { - defer testleak.AfterTest(c)() +func TestDAGPlanBuilderWindow(t *testing.T) { var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) vars := []string{ "set @@session.tidb_window_concurrency = 1", } - s.doTestDAGPlanBuilderWindow(c, vars, input, output) + doTestDAGPlanBuilderWindow(t, vars, input, output) } -func (s *testPlanSuite) TestDAGPlanBuilderWindowParallel(c *C) { - defer testleak.AfterTest(c)() +func TestDAGPlanBuilderWindowParallel(t *testing.T) { var input []string var output []struct { SQL string Best string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) vars := []string{ "set @@session.tidb_window_concurrency = 4", } - s.doTestDAGPlanBuilderWindow(c, vars, input, output) + doTestDAGPlanBuilderWindow(t, vars, input, output) } -func (s *testPlanSuite) TestTopNPushDownEmpty(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestTopNPushDownEmpty(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, index idx_a(a))") tk.MustQuery("select extract(day_hour from 'ziy') as res from t order by res limit 1").Check(testkit.Rows()) } -func (s *testPlanSuite) doTestDAGPlanBuilderWindow(c *C, vars, input []string, output []struct { +func doTestDAGPlanBuilderWindow(t *testing.T, vars, input []string, output []struct { SQL string Best string }) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") for _, v := range vars { - _, err = se.Execute(ctx, v) - c.Assert(err, IsNil) + tk.MustExec(v) } + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, tt := range input { - comment := Commentf("case:%v sql:%s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - - err = se.NewTxn(context.Background()) - c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql:%s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + + err = tk.Session().NewTxn(context.Background()) + require.NoError(t, err) + p, _, err := planner.Optimize(context.TODO(), tk.Session(), stmt, is) + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Best = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Best, comment) + require.Equal(t, output[i].Best, core.ToString(p), comment) } } -func (s *testPlanSuite) TestNominalSort(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestNominalSort(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") var input []string var output []struct { @@ -1769,88 +1586,65 @@ func (s *testPlanSuite) TestNominalSort(c *C) { tk.MustExec("insert into t values(1, 2)") tk.MustExec("insert into t values(2, 4)") tk.MustExec("insert into t values(3, 5)") - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestHintFromDiffDatabase(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.t1`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t1(a bigint, index idx_a(a));`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.t2(a bigint, index idx_a(a));`) - c.Assert(err, IsNil) - - _, err = se.Execute(ctx, "drop database if exists test2") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "create database test2") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, "use test2") - c.Assert(err, IsNil) +func TestHintFromDiffDatabase(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.t1`) + tk.MustExec(`create table test.t1(a bigint, index idx_a(a));`) + tk.MustExec(`create table test.t2(a bigint, index idx_a(a));`) + tk.MustExec("drop database if exists test2") + tk.MustExec("create database test2") + tk.MustExec("use test2") var input []string var output []struct { SQL string Plan string } - is := domain.GetDomain(se).InfoSchema() - s.testData.GetTestCases(c, &input, &output) + is := domain.GetDomain(tk.Session()).InfoSchema() + p := parser.New() + ctx := context.Background() + + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) - p, _, err := planner.Optimize(ctx, se, stmt, is) - c.Assert(err, IsNil, comment) - s.testData.OnRecord(func() { + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) + p, _, err := planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err, comment) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = core.ToString(p) }) - c.Assert(core.ToString(p), Equals, output[i].Plan, comment) + require.Equal(t, output[i].Plan, core.ToString(p), comment) } } -func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() - se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - ctx := context.Background() - _, err = se.Execute(ctx, "use test") - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `drop table if exists test.tt`) - c.Assert(err, IsNil) - _, err = se.Execute(ctx, `create table test.tt (a int,b int, index(a), index(b));`) - c.Assert(err, IsNil) - - _, err = se.Execute(ctx, "insert into tt values (1, 1), (2, 2), (3, 4)") - c.Assert(err, IsNil) - +func TestNthPlanHintWithExplain(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists test.tt`) + tk.MustExec(`create table test.tt (a int,b int, index(a), index(b));`) + tk.MustExec("insert into tt values (1, 1), (2, 2), (3, 4)") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) var input []string @@ -1858,23 +1652,24 @@ func (s *testPlanSuite) TestNthPlanHintWithExplain(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) } - // This assert makes sure a query with or without nth_plan() hint output exactly the same plan(including plan ID). + // This assertion makes sure a query with or without nth_plan() hint output exactly the same plan(including plan ID). // The query below is the same as queries in the testdata except for nth_plan() hint. - // Currently its output is the same as the second test case in the testdata, which is `output[1]`. If this doesn't + // Currently, its output is the same as the second test case in the testdata, which is `output[1]`. If this doesn't // hold in the future, you may need to modify this. tk.MustQuery("explain format = 'brief' select * from test.tt where a=1 and b=1").Check(testkit.Rows(output[1].Plan...)) } -func (s *testPlanSuite) TestEnumIndex(c *C) { +func TestEnumIndex(t *testing.T) { var ( input []string output []struct { @@ -1883,31 +1678,28 @@ func (s *testPlanSuite) TestEnumIndex(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(e enum('c','b','a',''), index idx(e))") tk.MustExec("insert ignore into t values(0),(1),(2),(3),(4);") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestIssue27233(c *C) { +func TestIssue27233(t *testing.T) { var ( input []string output []struct { @@ -1916,38 +1708,31 @@ func (s *testPlanSuite) TestIssue27233(c *C) { Result []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `PK_S_MULTI_31` (\n `COL1` tinyint(45) NOT NULL,\n `COL2` tinyint(45) NOT NULL,\n PRIMARY KEY (`COL1`,`COL2`) /*T![clustered_index] NONCLUSTERED */\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") tk.MustExec("insert into PK_S_MULTI_31 values(122,100),(124,-22),(124,34),(127,103);") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) - output[i].Result = s.testData.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Result = testdata.ConvertRowsToStrings(tk.MustQuery(ts).Sort().Rows()) }) tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) tk.MustQuery(ts).Sort().Check(testkit.Rows(output[i].Result...)) } } -func (s *testPlanSuite) TestPossibleProperties(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestPossibleProperties(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists student, sc") tk.MustExec("create table student(id int primary key auto_increment, name varchar(4) not null)") @@ -1959,7 +1744,7 @@ func (s *testPlanSuite) TestPossibleProperties(c *C) { )) } -func (s *testPlanSuite) TestSelectionPartialPushDown(c *C) { +func TestSelectionPartialPushDown(t *testing.T) { var ( input []string output []struct { @@ -1967,29 +1752,26 @@ func (s *testPlanSuite) TestSelectionPartialPushDown(c *C) { Plan []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int as (a+1) virtual)") tk.MustExec("create table t2(a int, b int as (a+1) virtual, c int, key idx_a(a))") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) }) tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testPlanSuite) TestIssue28316(c *C) { +func TestIssue28316(t *testing.T) { var ( input []string output []struct { @@ -1997,35 +1779,28 @@ func (s *testPlanSuite) TestIssue28316(c *C) { Plan []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) }) tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testPlanSuite) TestIssue30965(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestIssue30965(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t30965") tk.MustExec("CREATE TABLE `t30965` ( `a` int(11) DEFAULT NULL, `b` int(11) DEFAULT NULL, `c` int(11) DEFAULT NULL, `d` int(11) GENERATED ALWAYS AS (`a` + 1) VIRTUAL, KEY `ib` (`b`));") @@ -2041,7 +1816,7 @@ func (s *testPlanSuite) TestIssue30965(c *C) { " └─TableRowIDScan 10.00 cop[tikv] table:t30965 keep order:false, stats:pseudo")) } -func (s *testPlanSuite) TestMPPSinglePartitionType(c *C) { +func TestMPPSinglePartitionType(t *testing.T) { var ( input []string output []struct { @@ -2049,14 +1824,11 @@ func (s *testPlanSuite) TestMPPSinglePartitionType(c *C) { Plan []string } ) - s.testData.GetTestCases(c, &input, &output) - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists employee") tk.MustExec("create table employee(empid int, deptid int, salary decimal(10,2))") @@ -2064,7 +1836,7 @@ func (s *testPlanSuite) TestMPPSinglePartitionType(c *C) { is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "employee" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -2075,16 +1847,16 @@ func (s *testPlanSuite) TestMPPSinglePartitionType(c *C) { } for i, ts := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts }) if strings.HasPrefix(ts, "set") { tk.MustExec(ts) continue } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = ts - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) }) tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 5dc6b8a5a1ac4..2c9cbfabe4d0b 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -20,51 +20,28 @@ import ( "strings" "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - kit "github.com/pingcap/tidb/testkit" - "github.com/pingcap/tidb/util/israce" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/plancodec" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testPlanNormalize{}) - -type testPlanNormalize struct { - store kv.Storage - dom *domain.Domain - - testData testutil.TestData -} - -func (s *testPlanNormalize) SetUpSuite(c *C) { - testleak.BeforeTest() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - s.store = store - s.dom = dom - - s.testData, err = testutil.LoadTestSuiteData("testdata", "plan_normalized_suite") - c.Assert(err, IsNil) -} - -func (s *testPlanNormalize) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) - s.dom.Close() - s.store.Close() - testleak.AfterTest(c)() -} - -func (s *testPlanNormalize) TestPreferRangeScan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPreferRangeScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists test;") tk.MustExec("create table test(`id` int(10) NOT NULL AUTO_INCREMENT,`name` varchar(50) NOT NULL DEFAULT 'tidb',`age` int(11) NOT NULL,`addr` varchar(50) DEFAULT 'The ocean of stars',PRIMARY KEY (`id`),KEY `idx_age` (`age`))") @@ -86,33 +63,36 @@ func (s *testPlanNormalize) TestPreferRangeScan(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() + planNormalizedSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { if i == 0 { tk.MustExec("set session tidb_opt_prefer_range_scan=0") } else if i == 1 { tk.MustExec("set session tidb_opt_prefer_range_scan=1") } - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 tk.MustExec(tt) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(t, info) p, ok := info.Plan.(core.Plan) - c.Assert(ok, IsTrue) + require.True(t, ok) normalized, _ := core.NormalizePlan(p) normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) normalizedPlanRows := getPlanRows(normalizedPlan) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = normalizedPlanRows }) - compareStringSlice(c, normalizedPlanRows, output[i].Plan) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) } } -func (s *testPlanNormalize) TestNormalizedPlan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNormalizedPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='static';") tk.MustExec("drop table if exists t1,t2,t3,t4") @@ -125,35 +105,37 @@ func (s *testPlanNormalize) TestNormalizedPlan(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() + planNormalizedSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 tk.MustExec(tt) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(t, info) p, ok := info.Plan.(core.Plan) - c.Assert(ok, IsTrue) + require.True(t, ok) normalized, _ := core.NormalizePlan(p) normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) normalizedPlanRows := getPlanRows(normalizedPlan) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + require.NoError(t, err) + testdata.OnRecord(func() { output[i].SQL = tt output[i].Plan = normalizedPlanRows }) - compareStringSlice(c, normalizedPlanRows, output[i].Plan) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) } } -func (s *testPlanNormalize) TestNormalizedPlanForDiffStore(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNormalizedPlanForDiffStore(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int, b int, c int, primary key(a))") tk.MustExec("insert into t1 values(1,1,1), (2,2,2), (3,3,3)") - - tbl, err := s.dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) - c.Assert(err, IsNil) + tbl, err := dom.InfoSchema().TableByName(model.CIStr{O: "test", L: "test"}, model.CIStr{O: "t1", L: "t1"}) + require.NoError(t, err) // Set the hacked TiFlash replica for explain tests. tbl.Meta().TiFlashReplica = &model.TiFlashReplicaInfo{Count: 1, Available: true} @@ -162,78 +144,80 @@ func (s *testPlanNormalize) TestNormalizedPlanForDiffStore(c *C) { Digest string Plan []string } - s.testData.GetTestCases(c, &input, &output) + planNormalizedSuiteData := core.GetPlanNormalizedSuiteData() + planNormalizedSuiteData.GetTestCases(t, &input, &output) lastDigest := "" for i, tt := range input { - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 tk.MustExec(tt) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(t, info) ep, ok := info.Plan.(*core.Explain) - c.Assert(ok, IsTrue) + require.True(t, ok) normalized, digest := core.NormalizePlan(ep.TargetPlan) normalizedPlan, err := plancodec.DecodeNormalizedPlan(normalized) normalizedPlanRows := getPlanRows(normalizedPlan) - c.Assert(err, IsNil) - s.testData.OnRecord(func() { + require.NoError(t, err) + testdata.OnRecord(func() { output[i].Digest = digest.String() output[i].Plan = normalizedPlanRows }) - compareStringSlice(c, normalizedPlanRows, output[i].Plan) - c.Assert(digest.String() != lastDigest, IsTrue) + compareStringSlice(t, normalizedPlanRows, output[i].Plan) + require.NotEqual(t, digest.String(), lastDigest) lastDigest = digest.String() } } -func (s *testPlanNormalize) TestEncodeDecodePlan(c *C) { - if israce.RaceEnabled { - c.Skip("skip race test") - } - tk := testkit.NewTestKit(c, s.store) +func TestEncodeDecodePlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") tk.MustExec("create table t1 (a int key,b int,c int, index (b));") tk.MustExec("set tidb_enable_collect_execution_info=1;") - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 getPlanTree := func() string { - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(t, info) p, ok := info.Plan.(core.Plan) - c.Assert(ok, IsTrue) + require.True(t, ok) encodeStr := core.EncodePlan(p) planTree, err := plancodec.DecodePlan(encodeStr) - c.Assert(err, IsNil) + require.NoError(t, err) return planTree } tk.MustExec("select max(a) from t1 where a>0;") planTree := getPlanTree() - c.Assert(strings.Contains(planTree, "time"), IsTrue) - c.Assert(strings.Contains(planTree, "loops"), IsTrue) + require.True(t, strings.Contains(planTree, "time")) + require.True(t, strings.Contains(planTree, "loops")) tk.MustExec("insert into t1 values (1,1,1);") planTree = getPlanTree() - c.Assert(strings.Contains(planTree, "Insert"), IsTrue) - c.Assert(strings.Contains(planTree, "time"), IsTrue) - c.Assert(strings.Contains(planTree, "loops"), IsTrue) + require.True(t, strings.Contains(planTree, "Insert")) + require.True(t, strings.Contains(planTree, "time")) + require.True(t, strings.Contains(planTree, "loops")) tk.MustExec("with cte(a) as (select 1) select * from cte") planTree = getPlanTree() - c.Assert(strings.Contains(planTree, "CTE"), IsTrue) - c.Assert(strings.Contains(planTree, "1->Column#1"), IsTrue) - c.Assert(strings.Contains(planTree, "time"), IsTrue) - c.Assert(strings.Contains(planTree, "loops"), IsTrue) + require.True(t, strings.Contains(planTree, "CTE")) + require.True(t, strings.Contains(planTree, "1->Column#1")) + require.True(t, strings.Contains(planTree, "time")) + require.True(t, strings.Contains(planTree, "loops")) tk.MustExec("with cte(a) as (select 2) select * from cte") planTree = getPlanTree() - c.Assert(strings.Contains(planTree, "CTE"), IsTrue) - c.Assert(strings.Contains(planTree, "2->Column#1"), IsTrue) - c.Assert(strings.Contains(planTree, "time"), IsTrue) - c.Assert(strings.Contains(planTree, "loops"), IsTrue) + require.True(t, strings.Contains(planTree, "CTE")) + require.True(t, strings.Contains(planTree, "2->Column#1")) + require.True(t, strings.Contains(planTree, "time")) + require.True(t, strings.Contains(planTree, "loops")) } -func (s *testPlanNormalize) TestNormalizedDigest(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNormalizedDigest(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2,t3,t4, bmsql_order_line, bmsql_district,bmsql_stock") tk.MustExec("create table t1 (a int key,b int,c int, index (b));") @@ -289,10 +273,10 @@ func (s *testPlanNormalize) TestNormalizedDigest(c *C) { );`) err := failpoint.Enable("github.com/pingcap/tidb/planner/mockRandomPlanID", "return(true)") - c.Assert(err, IsNil) + require.NoError(t, err) defer func() { err = failpoint.Disable("github.com/pingcap/tidb/planner/mockRandomPlanID") - c.Assert(err, IsNil) + require.NoError(t, err) }() normalizedDigestCases := []struct { @@ -408,31 +392,31 @@ func (s *testPlanNormalize) TestNormalizedDigest(c *C) { }, } for _, testCase := range normalizedDigestCases { - testNormalizeDigest(tk, c, testCase.sql1, testCase.sql2, testCase.isSame) + testNormalizeDigest(tk, t, testCase.sql1, testCase.sql2, testCase.isSame) } } -func testNormalizeDigest(tk *testkit.TestKit, c *C, sql1, sql2 string, isSame bool) { +func testNormalizeDigest(tk *testkit.TestKit, t *testing.T, sql1, sql2 string, isSame bool) { tk.MustQuery(sql1) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(t, info) physicalPlan, ok := info.Plan.(core.PhysicalPlan) - c.Assert(ok, IsTrue) + require.True(t, ok) normalized1, digest1 := core.NormalizePlan(physicalPlan) tk.MustQuery(sql2) - info = tk.Se.ShowProcess() - c.Assert(info, NotNil) + info = tk.Session().ShowProcess() + require.NotNil(t, info) physicalPlan, ok = info.Plan.(core.PhysicalPlan) - c.Assert(ok, IsTrue) + require.True(t, ok) normalized2, digest2 := core.NormalizePlan(physicalPlan) - comment := Commentf("sql1: %v, sql2: %v\n%v !=\n%v\n", sql1, sql2, normalized1, normalized2) + comment := fmt.Sprintf("sql1: %v, sql2: %v\n%v !=\n%v\n", sql1, sql2, normalized1, normalized2) if isSame { - c.Assert(normalized1, Equals, normalized2, comment) - c.Assert(digest1.String(), Equals, digest2.String(), comment) + require.Equal(t, normalized1, normalized2, comment) + require.Equal(t, digest1.String(), digest2.String(), comment) } else { - c.Assert(normalized1 != normalized2, IsTrue, comment) - c.Assert(digest1.String() != digest2.String(), IsTrue, comment) + require.NotEqual(t, normalized1, normalized2, comment) + require.NotEqual(t, digest1.String(), digest2.String(), comment) } } @@ -441,15 +425,17 @@ func getPlanRows(planStr string) []string { return strings.Split(planStr, "\n") } -func compareStringSlice(c *C, ss1, ss2 []string) { - c.Assert(len(ss1), Equals, len(ss2)) +func compareStringSlice(t *testing.T, ss1, ss2 []string) { + require.Equal(t, len(ss1), len(ss2)) for i, s := range ss1 { - c.Assert(s, Equals, ss2[i]) + require.Equal(t, len(s), len(ss2[i])) } } -func (s *testPlanNormalize) TestExplainFormatHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplainFormatHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int not null, c2 int not null, key idx_c2(c2)) partition by range (c2) (partition p0 values less than (10), partition p1 values less than (20))") @@ -458,16 +444,17 @@ func (s *testPlanNormalize) TestExplainFormatHint(c *C) { "use_index(@`sel_2` `test`.`t2` `idx_c2`), hash_agg(@`sel_2`), use_index(@`sel_1` `test`.`t1` `idx_c2`), hash_agg(@`sel_1`)")) } -func (s *testPlanNormalize) TestExplainFormatHintRecoverableForTiFlashReplica(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestExplainFormatHintRecoverableForTiFlashReplica(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") // Create virtual `tiflash` replica info. - dom := domain.GetDomain(tk.Se) is := dom.InfoSchema() db, exists := is.SchemaByName(model.NewCIStr("test")) - c.Assert(exists, IsTrue) + require.True(t, exists) for _, tblInfo := range db.Tables { if tblInfo.Name.L == "t" { tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ @@ -478,18 +465,20 @@ func (s *testPlanNormalize) TestExplainFormatHintRecoverableForTiFlashReplica(c } rows := tk.MustQuery("explain select * from t").Rows() - c.Assert(rows[len(rows)-1][2], Equals, "cop[tiflash]") + require.Equal(t, rows[len(rows)-1][2], "cop[tiflash]") rows = tk.MustQuery("explain format='hint' select * from t").Rows() - c.Assert(rows[0][0], Equals, "read_from_storage(@`sel_1` tiflash[`test`.`t`])") + require.Equal(t, rows[0][0], "read_from_storage(@`sel_1` tiflash[`test`.`t`])") hints := tk.MustQuery("explain format='hint' select * from t;").Rows()[0][0] rows = tk.MustQuery(fmt.Sprintf("explain select /*+ %s */ * from t", hints)).Rows() - c.Assert(rows[len(rows)-1][2], Equals, "cop[tiflash]") + require.Equal(t, rows[len(rows)-1][2], "cop[tiflash]") } -func (s *testPlanNormalize) TestNthPlanHint(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestNthPlanHint(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tt") tk.MustExec("create table tt (a int,b int, index(a), index(b));") @@ -546,8 +535,10 @@ func (s *testPlanNormalize) TestNthPlanHint(c *C) { "Warning 1105 The parameter of nth_plan() is out of range.")) } -func (s *testPlanNormalize) BenchmarkDecodePlan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func BenchmarkDecodePlan(b *testing.B) { + store, clean := testkit.CreateMockStore(b) + defer clean() + tk := testkit.NewTestKit(b, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(10) key,b int);") @@ -562,24 +553,26 @@ func (s *testPlanNormalize) BenchmarkDecodePlan(c *C) { buf.WriteString(fmt.Sprintf("select count(1) as num,a from t where a='%v' group by a", i)) } query := buf.String() - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 tk.MustExec(query) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(b, info) p, ok := info.Plan.(core.PhysicalPlan) - c.Assert(ok, IsTrue) + require.True(b, ok) // TODO: optimize the encode plan performance when encode plan with runtimeStats - tk.Se.GetSessionVars().StmtCtx.RuntimeStatsColl = nil + tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl = nil encodedPlanStr := core.EncodePlan(p) - c.ResetTimer() - for i := 0; i < c.N; i++ { + b.ResetTimer() + for i := 0; i < b.N; i++ { _, err := plancodec.DecodePlan(encodedPlanStr) - c.Assert(err, IsNil) + require.NoError(b, err) } } -func (s *testPlanNormalize) BenchmarkEncodePlan(c *C) { - tk := testkit.NewTestKit(c, s.store) +func BenchmarkEncodePlan(b *testing.B) { + store, clean := testkit.CreateMockStore(b) + defer clean() + tk := testkit.NewTestKit(b, store) tk.MustExec("use test") tk.MustExec("drop table if exists th") tk.MustExec("set @@session.tidb_enable_table_partition = 1") @@ -588,25 +581,27 @@ func (s *testPlanNormalize) BenchmarkEncodePlan(c *C) { tk.MustExec("set @@tidb_slow_log_threshold=200000") query := "select count(*) from th t1 join th t2 join th t3 join th t4 join th t5 join th t6 where t1.i=t2.a and t1.i=t3.i and t3.i=t4.i and t4.i=t5.i and t5.i=t6.i" - tk.Se.GetSessionVars().PlanID = 0 + tk.Session().GetSessionVars().PlanID = 0 tk.MustExec(query) - info := tk.Se.ShowProcess() - c.Assert(info, NotNil) + info := tk.Session().ShowProcess() + require.NotNil(b, info) p, ok := info.Plan.(core.PhysicalPlan) - c.Assert(ok, IsTrue) - tk.Se.GetSessionVars().StmtCtx.RuntimeStatsColl = nil - c.ResetTimer() - for i := 0; i < c.N; i++ { + require.True(b, ok) + tk.Session().GetSessionVars().StmtCtx.RuntimeStatsColl = nil + b.ResetTimer() + for i := 0; i < b.N; i++ { core.EncodePlan(p) } } // Close issue 25729 -func (s *testPlanNormalize) TestIssue25729(c *C) { +func TestIssue25729(t *testing.T) { config.UpdateGlobal(func(conf *config.Config) { conf.Experimental.AllowsExpressionIndex = true }) - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists tt") // Case1 @@ -650,10 +645,10 @@ func (s *testPlanNormalize) TestIssue25729(c *C) { } func TestCopPaging(t *testing.T) { - store, clean := kit.CreateMockStore(t) + store, clean := testkit.CreateMockStore(t) defer clean() - tk := kit.NewTestKit(t, store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("set session tidb_enable_paging = 1") @@ -666,7 +661,7 @@ func TestCopPaging(t *testing.T) { // limit 960 should go paging for i := 0; i < 10; i++ { - tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(testkit.Rows( "Limit 4.00 root offset:0, count:960", "└─IndexLookUp 4.00 root paging:true", " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", @@ -677,19 +672,18 @@ func TestCopPaging(t *testing.T) { // selection between limit and indexlookup, limit 960 should also go paging for i := 0; i < 10; i++ { - tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(kit.Rows( + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 960").Check(testkit.Rows( "Limit 3.20 root offset:0, count:960", - "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", - " └─IndexLookUp 3.20 root paging:true", - " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", - " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", - " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", - " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + "└─IndexLookUp 3.20 root paging:true", + " ├─Selection(Build) 819.20 cop[tikv] gt(mod(test.t.id, 2), 0), le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) } // limit 961 exceeds the threshold, it should not go paging for i := 0; i < 10; i++ { - tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + tk.MustQuery("explain format='brief' select * from t force index(i) where id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(testkit.Rows( "Limit 4.00 root offset:0, count:961", "└─IndexLookUp 4.00 root ", " ├─Selection(Build) 1024.00 cop[tikv] le(test.t.id, 1024)", @@ -700,13 +694,158 @@ func TestCopPaging(t *testing.T) { // selection between limit and indexlookup, limit 961 should not go paging too for i := 0; i < 10; i++ { - tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(kit.Rows( + tk.MustQuery("explain format='brief' select * from t force index(i) where mod(id, 2) > 0 and id <= 1024 and c1 >= 0 and c1 <= 1024 and c2 in (2, 4, 6, 8) order by c1 limit 961").Check(testkit.Rows( "Limit 3.20 root offset:0, count:961", - "└─Selection 2.56 root gt(mod(test.t.id, 2), 0)", - " └─IndexLookUp 3.20 root ", - " ├─Selection(Build) 819.20 cop[tikv] le(test.t.id, 1024)", - " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", - " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", - " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + "└─IndexLookUp 3.20 root ", + " ├─Selection(Build) 819.20 cop[tikv] gt(mod(test.t.id, 2), 0), le(test.t.id, 1024)", + " │ └─IndexRangeScan 1024.00 cop[tikv] table:t, index:i(c1) range:[0,1024], keep order:true", + " └─Selection(Probe) 3.20 cop[tikv] in(test.t.c2, 2, 4, 6, 8)", + " └─TableRowIDScan 819.20 cop[tikv] table:t keep order:false")) + } +} + +func TestBuildFinalModeAggregation(t *testing.T) { + aggSchemaBuilder := func(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc) *expression.Schema { + schema := expression.NewSchema(make([]*expression.Column, 0, len(aggFuncs))...) + for _, agg := range aggFuncs { + newCol := &expression.Column{ + UniqueID: sctx.GetSessionVars().AllocPlanColumnID(), + RetType: agg.RetTp, + } + schema.Append(newCol) + } + return schema + } + isFinalAggMode := func(mode aggregation.AggFunctionMode) bool { + return mode == aggregation.FinalMode || mode == aggregation.CompleteMode + } + checkResult := func(sctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, groubyItems []expression.Expression) { + for partialIsCop := 0; partialIsCop < 2; partialIsCop++ { + for isMPPTask := 0; isMPPTask < 2; isMPPTask++ { + partial, final, _ := core.BuildFinalModeAggregation(sctx, &core.AggInfo{ + AggFuncs: aggFuncs, + GroupByItems: groubyItems, + Schema: aggSchemaBuilder(sctx, aggFuncs), + }, partialIsCop == 0, isMPPTask == 0) + if partial != nil { + for _, aggFunc := range partial.AggFuncs { + if partialIsCop == 0 { + require.True(t, !isFinalAggMode(aggFunc.Mode)) + } else { + require.True(t, isFinalAggMode(aggFunc.Mode)) + } + } + } + if final != nil { + for _, aggFunc := range final.AggFuncs { + require.True(t, isFinalAggMode(aggFunc.Mode)) + } + } + } + } + } + + ctx := core.MockContext() + + aggCol := &expression.Column{ + Index: 0, + RetType: types.NewFieldType(mysql.TypeLonglong), + } + gbyCol := &expression.Column{ + Index: 1, + RetType: types.NewFieldType(mysql.TypeLonglong), + } + orderCol := &expression.Column{ + Index: 2, + RetType: types.NewFieldType(mysql.TypeLonglong), + } + + emptyGroupByItems := make([]expression.Expression, 0, 1) + groupByItems := make([]expression.Expression, 0, 1) + groupByItems = append(groupByItems, gbyCol) + + orderByItems := make([]*util.ByItems, 0, 1) + orderByItems = append(orderByItems, &util.ByItems{ + Expr: orderCol, + Desc: true, + }) + + aggFuncs := make([]*aggregation.AggFuncDesc, 0, 5) + desc, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncMax, []expression.Expression{aggCol}, false) + require.NoError(t, err) + aggFuncs = append(aggFuncs, desc) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncFirstRow, []expression.Expression{aggCol}, false) + require.NoError(t, err) + aggFuncs = append(aggFuncs, desc) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncCount, []expression.Expression{aggCol}, false) + require.NoError(t, err) + aggFuncs = append(aggFuncs, desc) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncSum, []expression.Expression{aggCol}, false) + require.NoError(t, err) + aggFuncs = append(aggFuncs, desc) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncAvg, []expression.Expression{aggCol}, false) + require.NoError(t, err) + aggFuncs = append(aggFuncs, desc) + + aggFuncsWithDistinct := make([]*aggregation.AggFuncDesc, 0, 2) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncAvg, []expression.Expression{aggCol}, true) + require.NoError(t, err) + aggFuncsWithDistinct = append(aggFuncsWithDistinct, desc) + desc, err = aggregation.NewAggFuncDesc(ctx, ast.AggFuncCount, []expression.Expression{aggCol}, true) + require.NoError(t, err) + aggFuncsWithDistinct = append(aggFuncsWithDistinct, desc) + + groupConcatAggFuncs := make([]*aggregation.AggFuncDesc, 0, 4) + groupConcatWithoutDistinctWithoutOrderBy, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncGroupConcat, []expression.Expression{aggCol, aggCol}, false) + require.NoError(t, err) + groupConcatAggFuncs = append(groupConcatAggFuncs, groupConcatWithoutDistinctWithoutOrderBy) + groupConcatWithoutDistinctWithOrderBy, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncGroupConcat, []expression.Expression{aggCol, aggCol}, false) + require.NoError(t, err) + groupConcatWithoutDistinctWithOrderBy.OrderByItems = orderByItems + groupConcatAggFuncs = append(groupConcatAggFuncs, groupConcatWithoutDistinctWithOrderBy) + groupConcatWithDistinctWithoutOrderBy, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncGroupConcat, []expression.Expression{aggCol, aggCol}, true) + require.NoError(t, err) + groupConcatAggFuncs = append(groupConcatAggFuncs, groupConcatWithDistinctWithoutOrderBy) + groupConcatWithDistinctWithOrderBy, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncGroupConcat, []expression.Expression{aggCol, aggCol}, true) + require.NoError(t, err) + groupConcatWithDistinctWithOrderBy.OrderByItems = orderByItems + groupConcatAggFuncs = append(groupConcatAggFuncs, groupConcatWithDistinctWithOrderBy) + + // case 1 agg without distinct + checkResult(ctx, aggFuncs, emptyGroupByItems) + checkResult(ctx, aggFuncs, groupByItems) + + // case 2 agg with distinct + checkResult(ctx, aggFuncsWithDistinct, emptyGroupByItems) + checkResult(ctx, aggFuncsWithDistinct, groupByItems) + + // case 3 mixed with distinct and without distinct + mixedAggFuncs := make([]*aggregation.AggFuncDesc, 0, 10) + mixedAggFuncs = append(mixedAggFuncs, aggFuncs...) + mixedAggFuncs = append(mixedAggFuncs, aggFuncsWithDistinct...) + checkResult(ctx, mixedAggFuncs, emptyGroupByItems) + checkResult(ctx, mixedAggFuncs, groupByItems) + + // case 4 group concat + for _, groupConcatAggFunc := range groupConcatAggFuncs { + checkResult(ctx, []*aggregation.AggFuncDesc{groupConcatAggFunc}, emptyGroupByItems) + checkResult(ctx, []*aggregation.AggFuncDesc{groupConcatAggFunc}, groupByItems) + } + checkResult(ctx, groupConcatAggFuncs, emptyGroupByItems) + checkResult(ctx, groupConcatAggFuncs, groupByItems) + + // case 5 mixed group concat and other agg funcs + for _, groupConcatAggFunc := range groupConcatAggFuncs { + funcs := make([]*aggregation.AggFuncDesc, 0, 10) + funcs = append(funcs, groupConcatAggFunc) + funcs = append(funcs, aggFuncs...) + checkResult(ctx, funcs, emptyGroupByItems) + checkResult(ctx, funcs, groupByItems) + funcs = append(funcs, aggFuncsWithDistinct...) + checkResult(ctx, funcs, emptyGroupByItems) + checkResult(ctx, funcs, groupByItems) } + mixedAggFuncs = append(mixedAggFuncs, groupConcatAggFuncs...) + checkResult(ctx, mixedAggFuncs, emptyGroupByItems) + checkResult(ctx, mixedAggFuncs, groupByItems) } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 27056a8937c20..eb0db3cd97273 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -390,7 +390,9 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) var otherConditionsInJoin expression.CNFExprs var otherEqConditionsFromIn expression.CNFExprs - if p.JoinType == AntiSemiJoin { + /// For anti join, equal conditions from `in` clause requires additional processing, + /// for example, treat `null` as true. + if p.JoinType == AntiSemiJoin || p.JoinType == AntiLeftOuterSemiJoin || p.JoinType == LeftOuterSemiJoin { for _, condition := range p.OtherConditions { if expression.IsEQCondFromIn(condition) { otherEqConditionsFromIn = append(otherEqConditionsFromIn, condition) diff --git a/planner/core/plan_to_pb_test.go b/planner/core/plan_to_pb_test.go index 866f104887fef..e4267024c884f 100644 --- a/planner/core/plan_to_pb_test.go +++ b/planner/core/plan_to_pb_test.go @@ -28,6 +28,7 @@ import ( func TestColumnToProto(t *testing.T) { // Make sure the Flag is set in tipb.ColumnInfo + collate.SetNewCollationEnabledForTest(false) tp := types.NewFieldType(mysql.TypeLong) tp.Flag = 10 tp.Collate = "utf8_bin" @@ -59,7 +60,6 @@ func TestColumnToProto(t *testing.T) { require.Equal(t, int32(8), pc.Collation) collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) pc = util.ColumnToProto(col) expect = &tipb.ColumnInfo{ColumnId: 0, Tp: 3, Collation: -83, ColumnLen: -1, Decimal: -1, Flag: 10, Elems: []string(nil), DefaultVal: []uint8(nil), PkHandle: false, XXX_unrecognized: []uint8(nil)} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 4e940cd56a089..cf8cb52959f78 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2198,11 +2198,7 @@ func (b *PlanBuilder) genV2AnalyzeOptions( func (b *PlanBuilder) getSavedAnalyzeOpts(physicalID int64, tblInfo *model.TableInfo) (map[ast.AnalyzeOptionType]uint64, model.ColumnChoice, []*model.ColumnInfo, error) { analyzeOptions := map[ast.AnalyzeOptionType]uint64{} exec := b.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, "select sample_num,sample_rate,buckets,topn,column_choice,column_ids from mysql.analyze_options where table_id = %?", physicalID) - if err != nil { - return nil, model.DefaultChoice, nil, err - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, "select sample_num,sample_rate,buckets,topn,column_choice,column_ids from mysql.analyze_options where table_id = %?", physicalID) if err != nil { return nil, model.DefaultChoice, nil, err } @@ -2693,7 +2689,7 @@ func buildCleanupIndexFields() (*expression.Schema, types.NameSlice) { } func buildShowDDLJobsFields() (*expression.Schema, types.NameSlice) { - schema := newColumnsWithNames(11) + schema := newColumnsWithNames(12) schema.Append(buildColumnWithName("", "JOB_ID", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "DB_NAME", mysql.TypeVarchar, 64)) schema.Append(buildColumnWithName("", "TABLE_NAME", mysql.TypeVarchar, 64)) @@ -2702,6 +2698,7 @@ func buildShowDDLJobsFields() (*expression.Schema, types.NameSlice) { schema.Append(buildColumnWithName("", "SCHEMA_ID", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "TABLE_ID", mysql.TypeLonglong, 4)) schema.Append(buildColumnWithName("", "ROW_COUNT", mysql.TypeLonglong, 4)) + schema.Append(buildColumnWithName("", "CREATE_TIME", mysql.TypeDatetime, 19)) schema.Append(buildColumnWithName("", "START_TIME", mysql.TypeDatetime, 19)) schema.Append(buildColumnWithName("", "END_TIME", mysql.TypeDatetime, 19)) schema.Append(buildColumnWithName("", "STATE", mysql.TypeVarchar, 64)) @@ -2815,7 +2812,7 @@ type columnsWithNames struct { names types.NameSlice } -func newColumnsWithNames(cap int) *columnsWithNames { +func newColumnsWithNames(c int) *columnsWithNames { return &columnsWithNames{ cols: make([]*expression.Column, 0, 2), names: make(types.NameSlice, 0, 2), @@ -2880,7 +2877,17 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, // avoid to build Selection. show.Pattern = nil } - case ast.ShowTables, ast.ShowTableStatus: + case ast.ShowTables: + if p.DBName == "" { + return nil, ErrNoDB + } + var extractor ShowTablesTableExtractor + if extractor.Extract(show) { + p.Extractor = &extractor + // Avoid building Selection. + show.Pattern = nil + } + case ast.ShowTableStatus: if p.DBName == "" { return nil, ErrNoDB } @@ -3542,7 +3549,7 @@ func (b PlanBuilder) getInsertColExpr(ctx context.Context, insertPlan *Insert, m RetType: &x.Type, } case *driver.ParamMarkerExpr: - outExpr, err = expression.ParamMarkerExpression(b.ctx, x) + outExpr, err = expression.ParamMarkerExpression(b.ctx, x, false) default: b.curClause = fieldList // subquery in insert values should not reference upper scope diff --git a/planner/core/planbuilder_test.go b/planner/core/planbuilder_test.go index 05f8bf37e5838..f7b501a334258 100644 --- a/planner/core/planbuilder_test.go +++ b/planner/core/planbuilder_test.go @@ -19,10 +19,10 @@ import ( "fmt" "reflect" "strings" + "testing" "unsafe" _ "unsafe" // required by go:linkname - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" @@ -38,17 +38,10 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testPlanBuilderSuite{}) - -func (s *testPlanBuilderSuite) SetUpSuite(c *C) { -} - -type testPlanBuilderSuite struct { -} - -func (s *testPlanBuilderSuite) TestShow(c *C) { +func TestShow(t *testing.T) { node := &ast.ShowStmt{} tps := []ast.ShowStmtType{ ast.ShowEngines, @@ -78,12 +71,12 @@ func (s *testPlanBuilderSuite) TestShow(c *C) { node.Tp = tp schema, _ := buildShowSchema(node, false, false) for _, col := range schema.Columns { - c.Assert(col.RetType.Flen, Greater, 0) + require.Greater(t, col.RetType.Flen, 0) } } } -func (s *testPlanBuilderSuite) TestGetPathByIndexName(c *C) { +func TestGetPathByIndexName(t *testing.T) { tblInfo := &model.TableInfo{ Indices: make([]*model.IndexInfo, 0), PKIsHandle: true, @@ -97,15 +90,15 @@ func (s *testPlanBuilderSuite) TestGetPathByIndexName(c *C) { } path := getPathByIndexName(accessPath, model.NewCIStr("idx"), tblInfo) - c.Assert(path, NotNil) - c.Assert(path, Equals, accessPath[1]) + require.NotNil(t, path) + require.Equal(t, accessPath[1], path) path = getPathByIndexName(accessPath, model.NewCIStr("primary"), tblInfo) - c.Assert(path, NotNil) - c.Assert(path, Equals, accessPath[0]) + require.NotNil(t, path) + require.Equal(t, accessPath[0], path) path = getPathByIndexName(accessPath, model.NewCIStr("not exists"), tblInfo) - c.Assert(path, IsNil) + require.Nil(t, path) tblInfo = &model.TableInfo{ Indices: make([]*model.IndexInfo, 0), @@ -113,10 +106,10 @@ func (s *testPlanBuilderSuite) TestGetPathByIndexName(c *C) { } path = getPathByIndexName(accessPath, model.NewCIStr("primary"), tblInfo) - c.Assert(path, IsNil) + require.Nil(t, path) } -func (s *testPlanBuilderSuite) TestRewriterPool(c *C) { +func TestRewriterPool(t *testing.T) { builder, _ := NewPlanBuilder().Init(MockContext(), nil, &hint.BlockHintProcessor{}) // Make sure PlanBuilder.getExpressionRewriter() provides clean rewriter from pool. @@ -134,17 +127,17 @@ func (s *testPlanBuilderSuite) TestRewriterPool(c *C) { // Then, pick again and check if it's cleaned up. builder.rewriterCounter++ cleanRewriter := builder.getExpressionRewriter(context.TODO(), nil) - c.Assert(cleanRewriter, Equals, dirtyRewriter) // Rewriter should be reused. - c.Assert(cleanRewriter.asScalar, Equals, false) - c.Assert(cleanRewriter.aggrMap, IsNil) - c.Assert(cleanRewriter.preprocess, IsNil) - c.Assert(cleanRewriter.insertPlan, IsNil) - c.Assert(cleanRewriter.disableFoldCounter, Equals, 0) - c.Assert(len(cleanRewriter.ctxStack), Equals, 0) + require.Equal(t, dirtyRewriter, cleanRewriter) + require.Equal(t, false, cleanRewriter.asScalar) + require.Nil(t, cleanRewriter.aggrMap) + require.Nil(t, cleanRewriter.preprocess) + require.Nil(t, cleanRewriter.insertPlan) + require.Zero(t, cleanRewriter.disableFoldCounter) + require.Len(t, cleanRewriter.ctxStack, 0) builder.rewriterCounter-- } -func (s *testPlanBuilderSuite) TestDisableFold(c *C) { +func TestDisableFold(t *testing.T) { // Functions like BENCHMARK() shall not be folded into result 0, // but normal outer function with constant args should be folded. // Types of expression and first layer of args will be validated. @@ -165,31 +158,31 @@ func (s *testPlanBuilderSuite) TestDisableFold(c *C) { } ctx := MockContext() - for _, t := range cases { - st, err := parser.New().ParseOneStmt(t.SQL, "", "") - c.Assert(err, IsNil) + for _, c := range cases { + st, err := parser.New().ParseOneStmt(c.SQL, "", "") + require.NoError(t, err) stmt := st.(*ast.SelectStmt) expr := stmt.Fields.Fields[0].Expr builder, _ := NewPlanBuilder().Init(ctx, nil, &hint.BlockHintProcessor{}) builder.rewriterCounter++ rewriter := builder.getExpressionRewriter(context.TODO(), nil) - c.Assert(rewriter, NotNil) - c.Assert(rewriter.disableFoldCounter, Equals, 0) - rewritenExpression, _, err := builder.rewriteExprNode(rewriter, expr, true) - c.Assert(err, IsNil) - c.Assert(rewriter.disableFoldCounter, Equals, 0) // Make sure the counter is reduced to 0 in the end. + require.NotNil(t, rewriter) + require.Equal(t, 0, rewriter.disableFoldCounter) + rewrittenExpression, _, err := builder.rewriteExprNode(rewriter, expr, true) + require.NoError(t, err) + require.Equal(t, 0, rewriter.disableFoldCounter) builder.rewriterCounter-- - c.Assert(rewritenExpression, FitsTypeOf, t.Expected) - for i, expectedArg := range t.Args { - rewritenArg := expression.GetFuncArg(rewritenExpression, i) - c.Assert(rewritenArg, FitsTypeOf, expectedArg) + require.IsType(t, c.Expected, rewrittenExpression) + for i, expectedArg := range c.Args { + rewrittenArg := expression.GetFuncArg(rewrittenExpression, i) + require.IsType(t, expectedArg, rewrittenArg) } } } -func (s *testPlanBuilderSuite) TestDeepClone(c *C) { +func TestDeepClone(t *testing.T) { tp := types.NewFieldType(mysql.TypeLonglong) expr := &expression.Column{RetType: tp} byItems := []*util.ByItems{{Expr: expr}} @@ -199,24 +192,32 @@ func (s *testPlanBuilderSuite) TestDeepClone(c *C) { whiteList := []string{"*property.StatsInfo", "*sessionctx.Context", "*mock.Context"} return checkDeepClonedCore(reflect.ValueOf(p1), reflect.ValueOf(p2), typeName(reflect.TypeOf(p1)), whiteList, nil) } - c.Assert(checkDeepClone(sort1, sort2), ErrorMatches, "invalid slice pointers, path PhysicalSort.ByItems") + err := checkDeepClone(sort1, sort2) + require.Error(t, err) + require.Regexp(t, "invalid slice pointers, path PhysicalSort.ByItems", err.Error()) byItems2 := []*util.ByItems{{Expr: expr}} sort2.ByItems = byItems2 - c.Assert(checkDeepClone(sort1, sort2), ErrorMatches, "same pointer, path PhysicalSort.ByItems.*Expression") + err = checkDeepClone(sort1, sort2) + require.Error(t, err) + require.Regexp(t, "same pointer, path PhysicalSort.ByItems.*Expression", err.Error()) expr2 := &expression.Column{RetType: tp} byItems2[0].Expr = expr2 - c.Assert(checkDeepClone(sort1, sort2), ErrorMatches, "same pointer, path PhysicalSort.ByItems.*Expression.FieldType") + err = checkDeepClone(sort1, sort2) + require.Error(t, err) + require.Regexp(t, "same pointer, path PhysicalSort.ByItems.*Expression.FieldType", err.Error()) expr2.RetType = types.NewFieldType(mysql.TypeString) - c.Assert(checkDeepClone(sort1, sort2), ErrorMatches, "different values, path PhysicalSort.ByItems.*Expression.FieldType.uint8") + err = checkDeepClone(sort1, sort2) + require.Error(t, err) + require.Regexp(t, "different values, path PhysicalSort.ByItems.*Expression.FieldType.uint8", err.Error()) expr2.RetType = types.NewFieldType(mysql.TypeLonglong) - c.Assert(checkDeepClone(sort1, sort2), IsNil) + require.NoError(t, checkDeepClone(sort1, sort2)) } -func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { +func TestPhysicalPlanClone(t *testing.T) { ctx := mock.NewContext() col, cst := &expression.Column{RetType: types.NewFieldType(mysql.TypeString)}, &expression.Constant{RetType: types.NewFieldType(mysql.TypeLonglong)} stats := &property.StatsInfo{RowCount: 1000} @@ -225,9 +226,9 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { idxInfo := &model.IndexInfo{} hist := &statistics.Histogram{Bounds: chunk.New(nil, 0, 0)} aggDesc1, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncAvg, []expression.Expression{col}, false) - c.Assert(err, IsNil) + require.NoError(t, err) aggDesc2, err := aggregation.NewAggFuncDesc(ctx, ast.AggFuncCount, []expression.Expression{cst}, true) - c.Assert(err, IsNil) + require.NoError(t, err) aggDescs := []*aggregation.AggFuncDesc{aggDesc1, aggDesc2} // table scan @@ -238,7 +239,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { } tableScan = tableScan.Init(ctx, 0) tableScan.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(tableScan), IsNil) + require.NoError(t, checkPhysicalPlanClone(tableScan)) // table reader tableReader := &PhysicalTableReader{ @@ -247,7 +248,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { StoreType: kv.TiFlash, } tableReader = tableReader.Init(ctx, 0) - c.Assert(checkPhysicalPlanClone(tableReader), IsNil) + require.NoError(t, checkPhysicalPlanClone(tableReader)) // index scan indexScan := &PhysicalIndexScan{ @@ -259,7 +260,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { } indexScan = indexScan.Init(ctx, 0) indexScan.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(indexScan), IsNil) + require.NoError(t, checkPhysicalPlanClone(indexScan)) // index reader indexReader := &PhysicalIndexReader{ @@ -268,7 +269,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { OutputColumns: []*expression.Column{col, col}, } indexReader = indexReader.Init(ctx, 0) - c.Assert(checkPhysicalPlanClone(indexReader), IsNil) + require.NoError(t, checkPhysicalPlanClone(indexReader)) // index lookup indexLookup := &PhysicalIndexLookUpReader{ @@ -280,33 +281,33 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { PushedLimit: &PushedDownLimit{1, 2}, } indexLookup = indexLookup.Init(ctx, 0) - c.Assert(checkPhysicalPlanClone(indexLookup), IsNil) + require.NoError(t, checkPhysicalPlanClone(indexLookup)) // selection sel := &PhysicalSelection{Conditions: []expression.Expression{col, cst}} sel = sel.Init(ctx, stats, 0) - c.Assert(checkPhysicalPlanClone(sel), IsNil) + require.NoError(t, checkPhysicalPlanClone(sel)) // projection proj := &PhysicalProjection{Exprs: []expression.Expression{col, cst}} proj = proj.Init(ctx, stats, 0) - c.Assert(checkPhysicalPlanClone(proj), IsNil) + require.NoError(t, checkPhysicalPlanClone(proj)) // limit lim := &PhysicalLimit{Count: 1, Offset: 2} lim = lim.Init(ctx, stats, 0) - c.Assert(checkPhysicalPlanClone(lim), IsNil) + require.NoError(t, checkPhysicalPlanClone(lim)) // sort byItems := []*util.ByItems{{Expr: col}, {Expr: cst}} sort := &PhysicalSort{ByItems: byItems} sort = sort.Init(ctx, stats, 0) - c.Assert(checkPhysicalPlanClone(sort), IsNil) + require.NoError(t, checkPhysicalPlanClone(sort)) // topN topN := &PhysicalTopN{ByItems: byItems, Offset: 2333, Count: 2333} topN = topN.Init(ctx, stats, 0) - c.Assert(checkPhysicalPlanClone(topN), IsNil) + require.NoError(t, checkPhysicalPlanClone(topN)) // stream agg streamAgg := &PhysicalStreamAgg{basePhysicalAgg{ @@ -315,7 +316,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { }} streamAgg = streamAgg.initForStream(ctx, stats, 0) streamAgg.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(streamAgg), IsNil) + require.NoError(t, checkPhysicalPlanClone(streamAgg)) // hash agg hashAgg := &PhysicalHashAgg{basePhysicalAgg{ @@ -324,7 +325,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { }} hashAgg = hashAgg.initForHash(ctx, stats, 0) hashAgg.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(hashAgg), IsNil) + require.NoError(t, checkPhysicalPlanClone(hashAgg)) // hash join hashJoin := &PhysicalHashJoin{ @@ -333,7 +334,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { } hashJoin = hashJoin.Init(ctx, stats, 0) hashJoin.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(hashJoin), IsNil) + require.NoError(t, checkPhysicalPlanClone(hashJoin)) // merge join mergeJoin := &PhysicalMergeJoin{ @@ -342,7 +343,7 @@ func (s *testPlanBuilderSuite) TestPhysicalPlanClone(c *C) { } mergeJoin = mergeJoin.Init(ctx, stats, 0) mergeJoin.SetSchema(schema) - c.Assert(checkPhysicalPlanClone(mergeJoin), IsNil) + require.NoError(t, checkPhysicalPlanClone(mergeJoin)) } //go:linkname valueInterface reflect.valueInterface diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 5facc937e2609..9f3f2ce903e3e 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -40,11 +40,13 @@ import ( "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" tidbutil "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tracing" "github.com/pingcap/tipb/go-tipb" tikvstore "github.com/tikv/client-go/v2/kv" "go.uber.org/zap" @@ -61,9 +63,11 @@ type PointGetPlan struct { IndexInfo *model.IndexInfo PartitionInfo *model.PartitionDefinition Handle kv.Handle - HandleParam *driver.ParamMarkerExpr + HandleConstant *expression.Constant + handleFieldType *types.FieldType IndexValues []types.Datum - IndexValueParams []*driver.ParamMarkerExpr + IndexConstants []*expression.Constant + ColsFieldType []*types.FieldType IdxCols []*expression.Column IdxColLens []int AccessConditions []expression.Expression @@ -79,9 +83,10 @@ type PointGetPlan struct { } type nameValuePair struct { - colName string - value types.Datum - param *driver.ParamMarkerExpr + colName string + colFieldType *types.FieldType + value types.Datum + con *expression.Constant } // Schema implements the Plan interface. @@ -475,10 +480,17 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { switch x := node.(type) { case *ast.SelectStmt: defer func() { - if ctx.GetSessionVars().SelectLimit != math2.MaxUint64 && p != nil { + vars := ctx.GetSessionVars() + if vars.SelectLimit != math2.MaxUint64 && p != nil { ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("sql_select_limit is set, so point get plan is not activated")) p = nil } + if vars.StmtCtx.EnableOptimizeTrace && p != nil { + if vars.StmtCtx.OptimizeTracer == nil { + vars.StmtCtx.OptimizeTracer = &tracing.OptimizeTracer{} + } + vars.StmtCtx.OptimizeTracer.SetFastPlan(p.buildPlanTrace()) + } }() // Try to convert the `SELECT a, b, c FROM t WHERE (a, b, c) in ((1, 2, 4), (1, 3, 5))` to // `PhysicalUnionAll` which children are `PointGet` if exists an unique key (a, b, c) in table `t` @@ -870,7 +882,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool } pairs := make([]nameValuePair, 0, 4) - pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where) + pairs, isTableDual := getNameValuePairs(ctx, tbl, tblAlias, pairs, selStmt.Where) if pairs == nil && !isTableDual { return nil } @@ -908,7 +920,8 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.Handle = kv.IntHandle(handlePair.value.GetInt64()) p.UnsignedHandle = mysql.HasUnsignedFlag(fieldType.Flag) - p.HandleParam = handlePair.param + p.handleFieldType = fieldType + p.HandleConstant = handlePair.con p.PartitionInfo = partitionInfo return p } else if handlePair.value.Kind() != types.KindNull { @@ -941,7 +954,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool p.IsTableDual = true return p } - idxValues, idxValueParams := getIndexValues(idxInfo, pairs) + idxValues, idxConstant, colsFieldType := getIndexValues(idxInfo, pairs) if idxValues == nil { continue } @@ -960,7 +973,8 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.IndexInfo = idxInfo p.IndexValues = idxValues - p.IndexValueParams = idxValueParams + p.IndexConstants = idxConstant + p.ColsFieldType = colsFieldType p.PartitionInfo = partitionInfo if p.PartitionInfo != nil { p.partitionColumnPos = findPartitionIdx(idxInfo, pos, pairs) @@ -1145,42 +1159,58 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( +func getNameValuePairs(ctx sessionctx.Context, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( pairs []nameValuePair, isTableDual bool) { + stmtCtx := ctx.GetSessionVars().StmtCtx binOp, ok := expr.(*ast.BinaryOperationExpr) if !ok { return nil, false } if binOp.Op == opcode.LogicAnd { - nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.L) + nvPairs, isTableDual = getNameValuePairs(ctx, tbl, tblName, nvPairs, binOp.L) if nvPairs == nil || isTableDual { return nil, isTableDual } - nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.R) + nvPairs, isTableDual = getNameValuePairs(ctx, tbl, tblName, nvPairs, binOp.R) if nvPairs == nil || isTableDual { return nil, isTableDual } return nvPairs, isTableDual } else if binOp.Op == opcode.EQ { - var d types.Datum - var colName *ast.ColumnNameExpr - var param *driver.ParamMarkerExpr - var ok bool + var ( + d types.Datum + colName *ast.ColumnNameExpr + ok bool + con *expression.Constant + err error + ) if colName, ok = binOp.L.(*ast.ColumnNameExpr); ok { switch x := binOp.R.(type) { case *driver.ValueExpr: d = x.Datum case *driver.ParamMarkerExpr: - d = x.Datum - param = x + con, err = expression.ParamMarkerExpression(ctx, x, true) + if err != nil { + return nil, false + } + d, err = con.Eval(chunk.Row{}) + if err != nil { + return nil, false + } } } else if colName, ok = binOp.R.(*ast.ColumnNameExpr); ok { switch x := binOp.L.(type) { case *driver.ValueExpr: d = x.Datum case *driver.ParamMarkerExpr: - d = x.Datum - param = x + con, err = expression.ParamMarkerExpression(ctx, x, true) + if err != nil { + return nil, false + } + d, err = con.Eval(chunk.Row{}) + if err != nil { + return nil, false + } } } else { return nil, false @@ -1196,9 +1226,10 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, return nil, false } col := model.FindColumnInfo(tbl.Cols(), colName.Name.Name.L) - if col == nil || // Handling the case when the column is _tidb_rowid. - (col.Tp == mysql.TypeString && col.Collate == charset.CollationBin) { // This type we needn't to pad `\0` in here. - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), false + if col == nil { // Handling the case when the column is _tidb_rowid. + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: types.NewFieldType(mysql.TypeLonglong), value: d, con: con}), false + } else if col.Tp == mysql.TypeString && col.Collate == charset.CollationBin { // This type we needn't to pad `\0` in here. + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: &col.FieldType, value: d, con: con}), false } if !checkCanConvertInPointGet(col, d) { return nil, false @@ -1206,7 +1237,7 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) if err != nil { if terror.ErrorEqual(types.ErrOverflow, err) { - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), true + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: &col.FieldType, value: d, con: con}), true } // Some scenarios cast to int with error, but we may use this value in point get. if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { @@ -1218,7 +1249,7 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, if err != nil || cmp != 0 { return nil, false } - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), false + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, colFieldType: &col.FieldType, value: dVal, con: con}), false } return nil, false } @@ -1281,27 +1312,29 @@ func findPKHandle(tblInfo *model.TableInfo, pairs []nameValuePair) (handlePair n return handlePair, nil } -func getIndexValues(idxInfo *model.IndexInfo, pairs []nameValuePair) ([]types.Datum, []*driver.ParamMarkerExpr) { +func getIndexValues(idxInfo *model.IndexInfo, pairs []nameValuePair) ([]types.Datum, []*expression.Constant, []*types.FieldType) { idxValues := make([]types.Datum, 0, 4) - idxValueParams := make([]*driver.ParamMarkerExpr, 0, 4) + idxConstants := make([]*expression.Constant, 0, 4) + colsFieldType := make([]*types.FieldType, 0, 4) if len(idxInfo.Columns) != len(pairs) { - return nil, nil + return nil, nil, nil } if idxInfo.HasPrefixIndex() { - return nil, nil + return nil, nil, nil } for _, idxCol := range idxInfo.Columns { i := findInPairs(idxCol.Name.L, pairs) if i == -1 { - return nil, nil + return nil, nil, nil } idxValues = append(idxValues, pairs[i].value) - idxValueParams = append(idxValueParams, pairs[i].param) + idxConstants = append(idxConstants, pairs[i].con) + colsFieldType = append(colsFieldType, pairs[i].colFieldType) } if len(idxValues) > 0 { - return idxValues, idxValueParams + return idxValues, idxConstants, colsFieldType } - return nil, nil + return nil, nil, nil } func findInPairs(colName string, pairs []nameValuePair) int { diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index 300370b60dee6..ba83f6f0d384c 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -19,62 +19,37 @@ import ( "fmt" "math" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/kvcache" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" - "github.com/pingcap/tidb/util/testutil" dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" ) -var _ = SerialSuites(&testPointGetSuite{}) - -type testPointGetSuite struct { - store kv.Storage - dom *domain.Domain - testData testutil.TestData -} - -func (s *testPointGetSuite) SetUpSuite(c *C) { - testleak.BeforeTest() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - s.store = store - s.dom = dom - s.testData, err = testutil.LoadTestSuiteData("testdata", "point_get_plan") - c.Assert(err, IsNil) -} - -func (s *testPointGetSuite) TearDownSuite(c *C) { - s.dom.Close() - s.store.Close() - testleak.AfterTest(c)() - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) orgEnable := core.PreparedPlanCacheEnabled() defer func() { core.SetPreparedPlanCache(orgEnable) }() core.SetPreparedPlanCache(true) - var err error - tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + sess, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk.SetSession(sess) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -111,26 +86,26 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { tk.MustExec("set @param=1") tk.MustQuery("execute stmt1 using @param").Check(testkit.Rows("1 1 1")) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(0)) + require.Equal(t, float64(0), hit) tk.MustExec("set @param=2") tk.MustQuery("execute stmt1 using @param").Check(testkit.Rows("2 2 2")) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(1)) + require.Equal(t, float64(1), hit) tk.MustQuery("execute stmt2 using @param, @param").Check(testkit.Rows("2 2 2")) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(1)) + require.Equal(t, float64(1), hit) tk.MustExec("set @param=1") tk.MustQuery("execute stmt2 using @param, @param").Check(testkit.Rows("1 1 1")) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) // PointGetPlan for Update. tk.MustExec(`prepare stmt3 from "update t set b=b+1, c=c+1 where a = ?"`) tk.MustExec(`prepare stmt4 from "update t set a=a+1 where b = ? and c = ?"`) @@ -142,9 +117,9 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { "3 4 4", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) tk.MustExec("set @param=4") tk.MustExec("execute stmt4 using @param, @param") tk.MustQuery("select * from t").Check(testkit.Rows( @@ -153,9 +128,9 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { "4 4 4", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) // PointGetPlan for Delete. tk.MustExec(`prepare stmt5 from "delete from t where a = ?"`) tk.MustExec(`prepare stmt6 from "delete from t where b = ? and c = ?"`) @@ -165,18 +140,18 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { "2 2 2", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) tk.MustExec("set @param=2") tk.MustExec("execute stmt6 using @param, @param") tk.MustQuery("select * from t").Check(testkit.Rows( "1 1 1", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) tk.MustExec("insert into t (a, b, c) values (18446744073709551615, 4, 4)") tk.MustExec("set @p1=-1") tk.MustExec("set @p2=1") @@ -184,31 +159,35 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { tk.MustQuery("execute stmt7 using @p1").Check(testkit.Rows()) tk.MustQuery("execute stmt7 using @p2").Check(testkit.Rows("1")) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) hit = pb.GetCounter().GetValue() - c.Check(hit, Equals, float64(2)) + require.Equal(t, float64(2), hit) } -func (s *testPointGetSuite) TestPointGetForUpdate(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetForUpdate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table fu (id int primary key, val int)") tk.MustExec("insert into fu values (6, 6)") // In autocommit mode, outside a transaction, "for update" doesn't take effect. - checkUseForUpdate(tk, c, false) + checkUseForUpdate(tk, t, false) tk.MustExec("begin") - checkUseForUpdate(tk, c, true) + checkUseForUpdate(tk, t, true) tk.MustExec("rollback") tk.MustExec("set @@session.autocommit = 0") - checkUseForUpdate(tk, c, true) + checkUseForUpdate(tk, t, true) tk.MustExec("rollback") } -func (s *testPointGetSuite) TestPointGetForUpdateWithSubquery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetForUpdateWithSubquery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE users (id bigint(20) unsigned NOT NULL primary key, name longtext DEFAULT NULL, company_id bigint(20) DEFAULT NULL)") tk.MustExec("create table companies(id bigint primary key, name longtext default null)") @@ -220,18 +199,20 @@ func (s *testPointGetSuite) TestPointGetForUpdateWithSubquery(c *C) { tk.MustQuery("select * from users").Check(testkit.Rows("239 Company15 15")) } -func checkUseForUpdate(tk *testkit.TestKit, c *C, expectLock bool) { +func checkUseForUpdate(tk *testkit.TestKit, t *testing.T, expectLock bool) { res := tk.MustQuery("explain format = 'brief' select * from fu where id = 6 for update") // Point_Get_1 1.00 root table:fu, handle:6 opInfo := res.Rows()[0][4] selectLock := strings.Contains(fmt.Sprintf("%s", opInfo), "lock") - c.Assert(selectLock, Equals, expectLock) + require.Equal(t, expectLock, selectLock) tk.MustQuery("select * from fu where id = 6 for update").Check(testkit.Rows("6 6")) } -func (s *testPointGetSuite) TestWhereIn2BatchPointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestWhereIn2BatchPointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key auto_increment not null, b int, c int, unique key idx_abc(a, b, c))") @@ -317,34 +298,38 @@ func (s *testPointGetSuite) TestWhereIn2BatchPointGet(c *C) { } // Test that the plan id will be reset before optimization every time. -func (s *testPointGetSuite) TestPointGetId(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetId(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int primary key, c2 int)") defer tk.MustExec("drop table if exists t") pointGetQuery := "select c2 from t where c1 = 1" for i := 0; i < 2; i++ { - ctx := tk.Se.(sessionctx.Context) + ctx := tk.Session().(sessionctx.Context) stmts, err := session.Parse(ctx, pointGetQuery) - c.Assert(err, IsNil) - c.Assert(stmts, HasLen, 1) + require.NoError(t, err) + require.Len(t, stmts, 1) stmt := stmts[0] ret := &core.PreprocessorReturn{} err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) + require.NoError(t, err) p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) - c.Assert(err, IsNil) + require.NoError(t, err) // Test explain format = 'brief' result is useless, plan id will be reset when running `explain`. - c.Assert(p.ID(), Equals, 1) + require.Equal(t, 1, p.ID()) } } -func (s *testPointGetSuite) TestCBOPointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCBOPointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec("create table t (a varchar(20), b int, c int, d int, primary key(a), unique key(b, c))") tk.MustExec("insert into t values('1',4,4,1), ('2',3,3,2), ('3',2,2,3), ('4',1,1,4)") @@ -354,251 +339,256 @@ func (s *testPointGetSuite) TestCBOPointGet(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + pointGetPlanData := core.GetPointGetPlanData() + pointGetPlanData.GetTestCases(t, &input, &output) + require.Equal(t, len(input), len(output)) for i, sql := range input { plan := tk.MustQuery("explain format = 'brief' " + sql) res := tk.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(res.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) res.Check(testkit.Rows(output[i].Res...)) } } -func (s *testPointGetSuite) TestPartitionBatchPointGetPlanCache(c *C) { - testKit := testkit.NewTestKit(c, s.store) +func TestPartitionBatchPointGetPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) orgEnable := core.PreparedPlanCacheEnabled() defer func() { core.SetPreparedPlanCache(orgEnable) }() core.SetPreparedPlanCache(true) - - var err error - testKit.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + sess, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) - - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("create table t(a int, b int, unique key(a))") - testKit.MustExec("insert into t values(1,1),(2,2),(3,3)") - testKit.MustExec("prepare stmt from 'select * from t use index(a) where (a >= ? and a <= ?) or a = 3'") - testKit.MustExec("set @p=1,@q=2,@u=3") - testKit.MustQuery("execute stmt using @p,@p").Sort().Check(testkit.Rows( + require.NoError(t, err) + tk.SetSession(sess) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, unique key(a))") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("prepare stmt from 'select * from t use index(a) where (a >= ? and a <= ?) or a = 3'") + tk.MustExec("set @p=1,@q=2,@u=3") + tk.MustQuery("execute stmt using @p,@p").Sort().Check(testkit.Rows( "1 1", "3 3", )) - testKit.MustQuery("execute stmt using @u,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @u,@q").Sort().Check(testkit.Rows( "3 3", )) - testKit.MustExec("drop table t") - testKit.MustExec("create table t(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") - testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(2,1),(2,2),(2,3),(3,1),(3,2),(3,3)") - testKit.MustExec("set @@tidb_partition_prune_mode = 'static'") - testKit.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and b = ?'") - testKit.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( + tk.MustExec("drop table t") + tk.MustExec("create table t(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") + tk.MustExec("insert into t values(1,1),(1,2),(1,3),(2,1),(2,2),(2,3),(3,1),(3,2),(3,3)") + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and b = ?'") + tk.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( "2 2", )) - testKit.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( "1 1", "2 1", "3 1", )) - testKit.MustQuery("execute stmt using @u,@p,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @u,@p,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustExec("prepare stmt from 'select * from t where a in (?,?) and b = ?'") - testKit.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a in (?,?) and b = ?'") + tk.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@p,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@p,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", "2 2", )) - testKit.MustExec("prepare stmt from 'select * from t where a = ? and ((b >= ? and b <= ?) or b = 2)'") - testKit.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a = ? and ((b >= ? and b <= ?) or b = 2)'") + tk.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( "2 2", )) - testKit.MustQuery("execute stmt using @p,@p,@u").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@p,@u").Sort().Check(testkit.Rows( "1 1", "1 2", "1 3", )) - testKit.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustExec("prepare stmt from 'select * from t where a = ? and b in (?,?)'") - testKit.MustQuery("execute stmt using @p,@p,@q").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a = ? and b in (?,?)'") + tk.MustQuery("execute stmt using @p,@p,@q").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustQuery("execute stmt using @q,@p,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@p,@q").Sort().Check(testkit.Rows( "2 1", "2 2", )) - testKit.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn - testKit.MustExec("drop table t") - testKit.MustExec("create table t(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") - testKit.MustExec("insert into t values(1,1),(1,2),(1,3),(2,1),(2,2),(2,3),(3,1),(3,2),(3,3)") - testKit.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and b = ?'") - testKit.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.MustExec("drop table t") + tk.MustExec("create table t(a int, b int, primary key(a,b)) partition by hash(b) partitions 2") + tk.MustExec("insert into t values(1,1),(1,2),(1,3),(2,1),(2,2),(2,3),(3,1),(3,2),(3,3)") + tk.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and b = ?'") + tk.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( "2 2", )) - testKit.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( "1 1", "2 1", "3 1", )) - testKit.MustQuery("execute stmt using @u,@p,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @u,@p,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustExec("prepare stmt from 'select * from t where a in (?,?) and b = ?'") - testKit.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a in (?,?) and b = ?'") + tk.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@p,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@p,@p").Sort().Check(testkit.Rows( "1 1", "2 1", )) - testKit.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@p").Sort().Check(testkit.Rows( "2 1", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", "2 2", )) - testKit.MustExec("prepare stmt from 'select * from t where a = ? and ((b >= ? and b <= ?) or b = 2)'") - testKit.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a = ? and ((b >= ? and b <= ?) or b = 2)'") + tk.MustQuery("execute stmt using @p,@p,@p").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q,@q").Sort().Check(testkit.Rows( "2 2", )) - testKit.MustQuery("execute stmt using @p,@p,@u").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@p,@u").Sort().Check(testkit.Rows( "1 1", "1 2", "1 3", )) - testKit.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@u,@p").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustExec("prepare stmt from 'select * from t where a = ? and b in (?,?)'") - testKit.MustQuery("execute stmt using @p,@p,@q").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a = ? and b in (?,?)'") + tk.MustQuery("execute stmt using @p,@p,@q").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@p").Sort().Check(testkit.Rows( "1 1", "1 2", )) - testKit.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@q,@q").Sort().Check(testkit.Rows( "1 2", )) - testKit.MustQuery("execute stmt using @q,@p,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@p,@q").Sort().Check(testkit.Rows( "2 1", "2 2", )) - testKit.MustExec("drop table t") - testKit.MustExec("create table t(a int, b int, primary key(a)) partition by hash(a) partitions 2") - testKit.MustExec("insert into t values(1,0),(2,0),(3,0),(4,0)") - testKit.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and 1 = 1'") - testKit.MustQuery("execute stmt using @p,@p").Sort().Check(testkit.Rows( + tk.MustExec("drop table t") + tk.MustExec("create table t(a int, b int, primary key(a)) partition by hash(a) partitions 2") + tk.MustExec("insert into t values(1,0),(2,0),(3,0),(4,0)") + tk.MustExec("prepare stmt from 'select * from t where ((a >= ? and a <= ?) or a = 2) and 1 = 1'") + tk.MustQuery("execute stmt using @p,@p").Sort().Check(testkit.Rows( "1 0", "2 0", )) - testKit.MustQuery("execute stmt using @q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q").Sort().Check(testkit.Rows( "2 0", )) - testKit.MustQuery("execute stmt using @p,@u").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @p,@u").Sort().Check(testkit.Rows( "1 0", "2 0", "3 0", )) - testKit.MustQuery("execute stmt using @u,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @u,@p").Sort().Check(testkit.Rows( "2 0", )) - testKit.MustExec("prepare stmt from 'select * from t where a in (?,?) and 1 = 1'") - testKit.MustQuery("execute stmt using @p,@q").Sort().Check(testkit.Rows( + tk.MustExec("prepare stmt from 'select * from t where a in (?,?) and 1 = 1'") + tk.MustQuery("execute stmt using @p,@q").Sort().Check(testkit.Rows( "1 0", "2 0", )) - testKit.MustQuery("execute stmt using @q,@p").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@p").Sort().Check(testkit.Rows( "1 0", "2 0", )) - testKit.MustQuery("execute stmt using @q,@q").Sort().Check(testkit.Rows( + tk.MustQuery("execute stmt using @q,@q").Sort().Check(testkit.Rows( "2 0", )) } -func (s *testPointGetSuite) TestBatchPointGetPlanCache(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBatchPointGetPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) orgEnable := core.PreparedPlanCacheEnabled() defer func() { core.SetPreparedPlanCache(orgEnable) }() core.SetPreparedPlanCache(true) - var err error - tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + sess, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk.SetSession(sess) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -620,22 +610,23 @@ func (s *testPointGetSuite) TestBatchPointGetPlanCache(c *C) { )) } -func (s *testPointGetSuite) TestBatchPointGetPartition(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBatchPointGetPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) orgEnable := core.PreparedPlanCacheEnabled() defer func() { core.SetPreparedPlanCache(orgEnable) }() core.SetPreparedPlanCache(true) - var err error - tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + sess, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) - + require.NoError(t, err) + tk.SetSession(sess) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int) PARTITION BY HASH(a) PARTITIONS 4") tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (4, 4)") @@ -679,9 +670,11 @@ func (s *testPointGetSuite) TestBatchPointGetPartition(c *C) { tk.MustQuery("select * from t where (a, b) in ((1, 1), (2, 2), (3, 3), (4, 4))").Check(testkit.Rows()) } -func (s *testPointGetSuite) TestIssue19141(c *C) { +func TestIssue19141(t *testing.T) { // For issue 19141, fix partition selection on batch point get. - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t19141 (c_int int, primary key (c_int)) partition by hash ( c_int ) partitions 4") tk.MustExec("insert into t19141 values (1), (2), (3), (4)") @@ -698,8 +691,10 @@ func (s *testPointGetSuite) TestIssue19141(c *C) { tk.MustQuery("select * from t19141 order by c_int").Check(testkit.Rows("1", "2", "3", "4")) } -func (s *testPointGetSuite) TestSelectInMultiColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSelectInMultiColumns(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t2") tk.MustExec("create table t2(a int, b int, c int, primary key(a, b, c));") @@ -707,48 +702,46 @@ func (s *testPointGetSuite) TestSelectInMultiColumns(c *C) { tk.MustQuery("select * from t2 where (a, b, c) in ((1, 1, 1));").Check(testkit.Rows("1 1 1")) _, err := tk.Exec("select * from t2 where (a, b, c) in ((1, 1, 1, 1));") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1241]Operand should contain 3 column(s)") + require.Error(t, err) + require.Equal(t, "[expression:1241]Operand should contain 3 column(s)", err.Error()) _, err = tk.Exec("select * from t2 where (a, b, c) in ((1, 1, 1), (2, 2, 2, 2));") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1241]Operand should contain 3 column(s)") + require.Error(t, err) + require.Equal(t, "[expression:1241]Operand should contain 3 column(s)", err.Error()) _, err = tk.Exec("select * from t2 where (a, b, c) in ((1, 1), (2, 2, 2));") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[expression:1241]Operand should contain 3 column(s)") + require.Error(t, err) + require.Equal(t, "[expression:1241]Operand should contain 3 column(s)", err.Error()) } -func (s *testPointGetSuite) TestUpdateWithTableReadLockWillFail(c *C) { +func TestUpdateWithTableReadLockWillFail(t *testing.T) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.EnableTableLock = true }) - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table tbllock(id int, c int);") tk.MustExec("insert into tbllock values(1, 2), (2, 2);") tk.MustExec("lock table tbllock read;") _, err := tk.Exec("update tbllock set c = 3 where id = 2;") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[schema:1099]Table 'tbllock' was locked with a READ lock and can't be updated") + require.Error(t, err) + require.Equal(t, "[schema:1099]Table 'tbllock' was locked with a READ lock and can't be updated", err.Error()) } -func (s *testPointGetSuite) TestIssue20692(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue20692(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key, v int, vv int, vvv int, unique key u0(id, v, vv));") tk.MustExec("insert into t values(1, 1, 1, 1);") - se1, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - tk1 := testkit.NewTestKitWithSession(c, s.store, se1) - se2, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - tk2 := testkit.NewTestKitWithSession(c, s.store, se2) - se3, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - tk3 := testkit.NewTestKitWithSession(c, s.store, se3) + tk1 := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk3 := testkit.NewTestKit(t, store) tk1.MustExec("begin pessimistic;") tk1.MustExec("use test") tk2.MustExec("begin pessimistic;") @@ -769,7 +762,7 @@ func (s *testPointGetSuite) TestIssue20692(c *C) { // wait 50ms to ensure tk3 is blocked by tk2 select { case <-stop2: - c.Fail() + t.Fail() case <-time.After(50 * time.Millisecond): } @@ -779,8 +772,10 @@ func (s *testPointGetSuite) TestIssue20692(c *C) { tk3.MustQuery("select * from t;").Check(testkit.Rows("10 20 30 40")) } -func (s *testPointGetSuite) TestPointGetWithInvisibleIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetWithInvisibleIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int, unique(c1))") @@ -792,8 +787,10 @@ func (s *testPointGetSuite) TestPointGetWithInvisibleIndex(c *C) { )) } -func (s *testPointGetSuite) TestBatchPointGetWithInvisibleIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestBatchPointGetWithInvisibleIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int, unique(c1))") @@ -805,10 +802,12 @@ func (s *testPointGetSuite) TestBatchPointGetWithInvisibleIndex(c *C) { )) } -func (s *testPointGetSuite) TestCBOShouldNotUsePointGet(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestCBOShouldNotUsePointGet(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop tables if exists t1, t2, t3, t4, t5") tk.MustExec("create table t1(id varchar(20) primary key)") tk.MustExec("create table t2(id varchar(20), unique(id))") @@ -827,22 +826,27 @@ func (s *testPointGetSuite) TestCBOShouldNotUsePointGet(c *C) { Plan []string Res []string } - s.testData.GetTestCases(c, &input, &output) + + pointGetPlanData := core.GetPointGetPlanData() + pointGetPlanData.GetTestCases(t, &input, &output) + require.Equal(t, len(input), len(output)) for i, sql := range input { plan := tk.MustQuery("explain format = 'brief' " + sql) res := tk.MustQuery(sql) - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = sql - output[i].Plan = s.testData.ConvertRowsToStrings(plan.Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(res.Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(plan.Rows()) + output[i].Res = testdata.ConvertRowsToStrings(res.Rows()) }) plan.Check(testkit.Rows(output[i].Plan...)) res.Check(testkit.Rows(output[i].Res...)) } } -func (s *testPointGetSuite) TestPointGetWithIndexHints(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestPointGetWithIndexHints(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") // point get @@ -887,20 +891,24 @@ func (s *testPointGetSuite) TestPointGetWithIndexHints(c *C) { " └─TableRowIDScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo")) } -func (s *testPointGetSuite) TestIssue18042(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue18042(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, primary key(a), index ab(a, b));") tk.MustExec("insert into t values (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4)") tk.MustExec("SELECT /*+ MAX_EXECUTION_TIME(100), MEMORY_QUOTA(1 MB) */ * FROM t where a = 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemQuotaQuery, Equals, int64(1<<20)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MaxExecutionTime, Equals, uint64(100)) + require.Equal(t, int64(1<<20), tk.Session().GetSessionVars().StmtCtx.MemQuotaQuery) + require.Equal(t, uint64(100), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) tk.MustExec("drop table t") } -func (s *testPointGetSuite) TestIssue26638(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue26638(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a float, unique index uidx(a));") @@ -929,8 +937,10 @@ func (s *testPointGetSuite) TestIssue26638(c *C) { tk.MustQuery("execute stmt4 using @i,@e,@d,@a,@b,@c,@e,@g,@h;").Check(testkit.Rows("0 1 2")) } -func (s *testPointGetSuite) TestIssue23511(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestIssue23511(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2;") tk.MustExec("CREATE TABLE `t1` (`COL1` bit(11) NOT NULL,PRIMARY KEY (`COL1`));") diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 6b336384ef8da..d7f754771a64e 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -21,54 +21,40 @@ import ( "math/rand" "strconv" "strings" + "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/kvcache" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" "github.com/prometheus/client_golang/prometheus" dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testPrepareSuite{}) -var _ = SerialSuites(&testPrepareSerialSuite{}) - -type testPrepareSuite struct { -} - -type testPrepareSerialSuite struct { -} - -func (s *testPrepareSerialSuite) TestRandomFlushPlanCache(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - tk2 := testkit.NewTestKit(c, store) +func TestRandomFlushPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -97,7 +83,7 @@ func (s *testPrepareSerialSuite) TestRandomFlushPlanCache(c *C) { rand.Seed(time.Now().Unix()) for i := 0; i < 10; i++ { - // Warm up to make sure all of the plans are in the cache. + // Warm up to make sure all the plans are in the cache. for _, execStmt := range execStmts { tk.MustExec(execStmt) tk.MustExec(execStmt) @@ -119,15 +105,15 @@ func (s *testPrepareSerialSuite) TestRandomFlushPlanCache(c *C) { } else { err = tk2.ExecToErr("admin flush instance plan_cache;") } - c.Check(err, Equals, nil) + require.NoError(t, err) } else if randNum == 1 { session1PC = "0" err = tk.ExecToErr("admin flush session plan_cache;") - c.Check(err, Equals, nil) + require.NoError(t, err) } else if randNum == 2 { session2PC = "0" err = tk2.ExecToErr("admin flush session plan_cache;") - c.Check(err, Equals, nil) + require.NoError(t, err) } for _, execStmt := range execStmts { @@ -140,31 +126,25 @@ func (s *testPrepareSerialSuite) TestRandomFlushPlanCache(c *C) { } err = tk.ExecToErr("admin flush instance plan_cache;") - c.Check(err, Equals, nil) + require.NoError(t, err) } err = tk.ExecToErr("admin flush global plan_cache;") - c.Check(err.Error(), Equals, "Do not support the 'admin flush global scope.'") + require.EqualError(t, err, "Do not support the 'admin flush global scope.'") } -func (s *testPrepareSerialSuite) TestFlushPlanCache(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - tk2 := testkit.NewTestKit(c, store) +func TestFlushPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -244,27 +224,21 @@ func (s *testPrepareSerialSuite) TestFlushPlanCache(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) err = tk.ExecToErr("admin flush global plan_cache;") - c.Check(err.Error(), Equals, "Do not support the 'admin flush global scope.'") + require.EqualError(t, err, "Do not support the 'admin flush global scope.'") } -func (s *testPrepareSerialSuite) TestFlushPlanCacheWithoutPCEnable(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - tk2 := testkit.NewTestKit(c, store) +func TestFlushPlanCacheWithoutPCEnable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(false) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + tk2 := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -346,26 +320,20 @@ func (s *testPrepareSerialSuite) TestFlushPlanCacheWithoutPCEnable(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) err = tk.ExecToErr("admin flush global plan_cache;") - c.Check(err.Error(), Equals, "Do not support the 'admin flush global scope.'") + require.EqualError(t, err, "Do not support the 'admin flush global scope.'") } -func (s *testPrepareSerialSuite) TestPrepareCache(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -394,7 +362,7 @@ func (s *testPrepareSerialSuite) TestPrepareCache(c *C) { tk.MustQuery("execute stmt6").Check(testkit.Rows("1", "2", "3", "4", "5", "6")) // test privilege change - rootSe := tk.Se + rootSe := tk.Session() tk.MustExec("drop table if exists tp") tk.MustExec(`create table tp(c1 int, c2 int, primary key (c1))`) tk.MustExec(`insert into tp values(1, 1), (2, 2), (3, 3)`) @@ -403,56 +371,50 @@ func (s *testPrepareSerialSuite) TestPrepareCache(c *C) { tk.MustExec(`grant select on test.tp to u_tp@'localhost';`) // user u_tp - userSess := newSession(c, store, "test") - c.Assert(userSess.Auth(&auth.UserIdentity{Username: "u_tp", Hostname: "localhost"}, nil, nil), IsTrue) - mustExec(c, userSess, `prepare ps_stp_r from 'select * from tp where c1 > ?'`) - mustExec(c, userSess, `set @p2 = 2`) - tk.Se = userSess + userSess := newSession(t, store, "test") + require.True(t, userSess.Auth(&auth.UserIdentity{Username: "u_tp", Hostname: "localhost"}, nil, nil)) + mustExec(t, userSess, `prepare ps_stp_r from 'select * from tp where c1 > ?'`) + mustExec(t, userSess, `set @p2 = 2`) + tk.SetSession(userSess) tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) // root revoke - tk.Se = rootSe + tk.SetSession(rootSe) tk.MustExec(`revoke all on test.tp from 'u_tp'@'localhost';`) // user u_tp - tk.Se = userSess + tk.SetSession(userSess) _, err = tk.Exec(`execute ps_stp_r using @p2`) - c.Assert(err, NotNil) + require.Error(t, err) // grant again - tk.Se = rootSe + tk.SetSession(rootSe) tk.MustExec(`grant select on test.tp to u_tp@'localhost';`) // user u_tp - tk.Se = userSess + tk.SetSession(userSess) tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) tk.MustQuery(`execute ps_stp_r using @p2`).Check(testkit.Rows("3 3")) // restore - tk.Se = rootSe + tk.SetSession(rootSe) tk.MustExec("drop table if exists tp") tk.MustExec(`DROP USER 'u_tp'@'localhost';`) } -func (s *testPrepareSerialSuite) TestPrepareCacheIndexScan(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheIndexScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -547,23 +509,17 @@ func randValue(tk *testkit.TestKit, tbl, col, dtype, rtype string) string { return "'invalid-type-" + dtype + "'" } -func (s *testPrepareSerialSuite) TestPrepareCacheChangingParamType(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheChangingParamType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t_tinyint, t_unsigned, t_float, t_decimal, t_year`) @@ -599,11 +555,10 @@ func (s *testPrepareSerialSuite) TestPrepareCacheChangingParamType(c *C) { compareResult := func(sql1, sql2 string) { raw, err := tk.Exec(sql1) if err != nil { - err := tk.ExecToErr(sql2) - c.Assert(err, NotNil) + require.Error(t, tk.ExecToErr(sql2)) return } - rs := tk.ResultSetToResult(raw, Commentf("sql1:%s, sql2:%v", sql1, sql2)) + rs := tk.ResultSetToResult(raw, fmt.Sprintf("sql1:%s, sql2:%v", sql1, sql2)) rs.Sort().Check(tk.MustQuery(sql2).Sort().Rows()) } @@ -619,23 +574,17 @@ func (s *testPrepareSerialSuite) TestPrepareCacheChangingParamType(c *C) { } } -func (s *testPrepareSerialSuite) TestPrepareCacheChangeCharsetCollation(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheChangeCharsetCollation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t`) @@ -666,23 +615,17 @@ func (s *testPrepareSerialSuite) TestPrepareCacheChangeCharsetCollation(c *C) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheDeferredFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -698,81 +641,71 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") ctx := context.TODO() + p := parser.New() + p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) for i := 0; i < 2; i++ { - stmt, err := s.ParseOneStmt(sql1, "", "") - c.Check(err, IsNil) - is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) - builder, _ := core.NewPlanBuilder().Init(tk.Se, is, &hint.BlockHintProcessor{}) + stmt, err := p.ParseOneStmt(sql1, "", "") + require.NoError(t, err) + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) + builder, _ := core.NewPlanBuilder().Init(tk.Session(), is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) - c.Check(err, IsNil) + require.NoError(t, err) execPlan, ok := p.(*core.Execute) - c.Check(ok, IsTrue) - err = executor.ResetContextOfStmt(tk.Se, stmt) - c.Assert(err, IsNil) - err = execPlan.OptimizePreparedPlan(ctx, tk.Se, is) - c.Check(err, IsNil) + require.True(t, ok) + err = executor.ResetContextOfStmt(tk.Session(), stmt) + require.NoError(t, err) + err = execPlan.OptimizePreparedPlan(ctx, tk.Session(), is) + require.NoError(t, err) planStr[i] = core.ToString(execPlan.Plan) - c.Check(planStr[i], Matches, expectedPattern, Commentf("for %dth %s", i, sql1)) + require.Regexpf(t, expectedPattern, planStr[i], "for %dth %s", i, sql1) pb := &dto.Metric{} err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt[i] = pb.GetCounter().GetValue() - c.Check(cnt[i], Equals, float64(i)) + require.Equal(t, float64(i), cnt[i]) time.Sleep(time.Millisecond * 10) } - c.Assert(planStr[0] < planStr[1], IsTrue, Commentf("plan 1: %v, plan 2: %v", planStr[0], planStr[1])) + require.Lessf(t, planStr[0], planStr[1], "plan 1: %v, plan 2: %v", planStr[0], planStr[1]) } -func (s *testPrepareSerialSuite) TestPrepareCacheNow(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheNow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec(`prepare stmt1 from "select now(), current_timestamp(), utc_timestamp(), unix_timestamp(), sleep(0.1), now(), current_timestamp(), utc_timestamp(), unix_timestamp()"`) // When executing one statement at the first time, we don't usTestPrepareCacheDeferredFunctione cache, so we need to execute it at least twice to test the cache. _ = tk.MustQuery("execute stmt1").Rows() rs := tk.MustQuery("execute stmt1").Rows() - c.Assert(rs[0][0].(string), Equals, rs[0][5].(string)) - c.Assert(rs[0][1].(string), Equals, rs[0][6].(string)) - c.Assert(rs[0][2].(string), Equals, rs[0][7].(string)) - c.Assert(rs[0][3].(string), Equals, rs[0][8].(string)) + require.Equal(t, rs[0][5].(string), rs[0][0].(string)) + require.Equal(t, rs[0][6].(string), rs[0][1].(string)) + require.Equal(t, rs[0][7].(string), rs[0][2].(string)) + require.Equal(t, rs[0][8].(string), rs[0][3].(string)) } -func (s *testPrepareSerialSuite) TestPrepareOverMaxPreparedStmtCount(c *C) { - c.Skip("unstable, skip it and fix it before 20210705") - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() +func TestPrepareOverMaxPreparedStmtCount(t *testing.T) { + t.Skip("unstable, skip it and fix it before 20210705") + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") // test prepare and deallocate. prePrepared := readGaugeInt(metrics.PreparedStmtGauge) tk.MustExec(`prepare stmt1 from "select 1"`) onePrepared := readGaugeInt(metrics.PreparedStmtGauge) - c.Assert(prePrepared+1, Equals, onePrepared) + require.Equal(t, onePrepared, prePrepared+1) tk.MustExec(`deallocate prepare stmt1`) deallocPrepared := readGaugeInt(metrics.PreparedStmtGauge) - c.Assert(prePrepared, Equals, deallocPrepared) + require.Equal(t, deallocPrepared, prePrepared) // test change global limit and make it affected in test session. tk.MustQuery("select @@max_prepared_stmt_count").Check(testkit.Rows("-1")) @@ -782,45 +715,38 @@ func (s *testPrepareSerialSuite) TestPrepareOverMaxPreparedStmtCount(c *C) { // test close session to give up all prepared stmt tk.MustExec(`prepare stmt2 from "select 1"`) prePrepared = readGaugeInt(metrics.PreparedStmtGauge) - tk.Se.Close() + tk.Session().Close() drawPrepared := readGaugeInt(metrics.PreparedStmtGauge) - c.Assert(prePrepared-1, Equals, drawPrepared) + require.Equal(t, drawPrepared, prePrepared-1) // test meet max limit. - tk.Se = nil + tk.RefreshSession() tk.MustQuery("select @@max_prepared_stmt_count").Check(testkit.Rows("2")) for i := 1; ; i++ { prePrepared = readGaugeInt(metrics.PreparedStmtGauge) if prePrepared >= 2 { - _, err = tk.Exec(`prepare stmt` + strconv.Itoa(i) + ` from "select 1"`) - c.Assert(terror.ErrorEqual(err, variable.ErrMaxPreparedStmtCountReached), IsTrue) + _, err := tk.Exec(`prepare stmt` + strconv.Itoa(i) + ` from "select 1"`) + require.True(t, terror.ErrorEqual(err, variable.ErrMaxPreparedStmtCountReached)) break } - _, err = tk.Exec(`prepare stmt` + strconv.Itoa(i) + ` from "select 1"`) - c.Assert(err, IsNil) + _, err := tk.Exec(`prepare stmt` + strconv.Itoa(i) + ` from "select 1"`) + require.NoError(t, err) } } // unit test for issue https://github.com/pingcap/tidb/issues/8518 -func (s *testPrepareSerialSuite) TestPrepareTableAsNameOnGroupByWithCache(c *C) { - c.Skip("unstable, skip it and fix it before 20210702") - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareTableAsNameOnGroupByWithCache(t *testing.T) { + t.Skip("unstable, skip it and fix it before 20210702") + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -844,23 +770,17 @@ func (s *testPrepareSerialSuite) TestPrepareTableAsNameOnGroupByWithCache(c *C) tk.MustQuery("execute stmt").Sort().Check(testkit.Rows("partner1", "partner2", "partner3", "partner4")) } -func (s *testPrepareSerialSuite) TestPrepareCachePointGetInsert(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCachePointGetInsert(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -886,23 +806,17 @@ func (s *testPrepareSerialSuite) TestPrepareCachePointGetInsert(c *C) { tk.MustQuery("select * from t2 order by a").Check(testkit.Rows("1 1", "2 2", "3 3")) } -func (s *testPrepareSerialSuite) TestIssue31280(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue31280(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists UK_MU15569;") @@ -921,36 +835,30 @@ func (s *testPrepareSerialSuite) TestIssue31280(c *C) { tk.MustExec("set @a=-32373, @b='545:50:46.85487';") // The tableDual plan can not be cached. res := tk.MustQuery("execute stmt using @a,@b;") - c.Assert(len(res.Rows()), Equals, 0) + require.Len(t, res.Rows(), 0) tk.MustExec("set @a=-27225, @b='-836:46:08';") res = tk.MustQuery("execute stmt using @a,@b;") - c.Assert(len(res.Rows()), Equals, 1) + require.Len(t, res.Rows(), 1) tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) tk.MustQuery("execute stmt using @a,@b;") tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) res1 := tk.MustQuery("select * from UK_MU15569 where col2 >= -27225 and col1 is not null and col3 = '-836:46:08';") - c.Assert(res.Rows(), DeepEquals, res1.Rows()) + require.Equal(t, res1.Rows(), res.Rows()) } -func (s *testPrepareSerialSuite) TestIssue31375(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue31375(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists IDT_MULTI15844STROBJSTROBJ;") @@ -991,20 +899,12 @@ func readGaugeInt(g prometheus.Gauge) int { } // unit test for issue https://github.com/pingcap/tidb/issues/9478 -func (s *testPrepareSuite) TestPrepareWithWindowFunction(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() +func TestPrepareWithWindowFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_enable_window_function = 1") - defer func() { - tk.MustExec("set @@tidb_enable_window_function = 0") - }() + defer tk.MustExec("set @@tidb_enable_window_function = 0") tk.MustExec("use test") tk.MustExec("create table window_prepare(a int, b double)") tk.MustExec("insert into window_prepare values(1, 1.1), (2, 1.9)") @@ -1017,17 +917,10 @@ func (s *testPrepareSuite) TestPrepareWithWindowFunction(c *C) { tk.MustQuery("execute stmt2 using @a, @b").Check(testkit.Rows("0", "0")) } -func (s *testPrepareSuite) TestPrepareWithSnapshot(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() - +func TestPrepareWithSnapshot(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) safePointName := "tikv_gc_safe_point" safePointValue := "20060102-15:04:05 -0700" safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" @@ -1051,16 +944,10 @@ func (s *testPrepareSuite) TestPrepareWithSnapshot(c *C) { tk.MustQuery("execute s2").Check(testkit.Rows("1 2")) } -func (s *testPrepareSuite) TestPrepareForGroupByItems(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() +func TestPrepareForGroupByItems(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(id int, v int)") @@ -1071,21 +958,16 @@ func (s *testPrepareSuite) TestPrepareForGroupByItems(c *C) { tk.MustExec("prepare s1 from 'select max(v) from t group by ?';") tk.MustExec("set @a=2;") - err = tk.ExecToErr("execute s1 using @a;") - c.Assert(err.Error(), Equals, "Unknown column '2' in 'group statement'") + err := tk.ExecToErr("execute s1 using @a;") + require.EqualError(t, err, "Unknown column '2' in 'group statement'") tk.MustExec("set @a=2.0;") tk.MustQuery("execute s1 using @a;").Check(testkit.Rows("3")) } -func (s *testPrepareSuite) TestPrepareCacheForClusteredIndex(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - }() +func TestPrepareCacheForClusteredIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t1(k varchar(100) primary key clustered, v1 int, v2 int)") tk.MustExec("insert into t1 (k, v1, v2) values('a', 1, 2), ('b', 1, 1)") @@ -1096,24 +978,17 @@ func (s *testPrepareSuite) TestPrepareCacheForClusteredIndex(c *C) { tk.MustQuery("execute prepare_1").Check(testkit.Rows("2 100", "1 ")) } -func (s *testPrepareSerialSuite) TestPrepareCacheForPartition(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheForPartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") for _, val := range []string{string(variable.Static), string(variable.Dynamic)} { @@ -1221,37 +1096,30 @@ func (s *testPrepareSerialSuite) TestPrepareCacheForPartition(c *C) { } } -func newSession(c *C, store kv.Storage, dbName string) session.Session { +func newSession(t *testing.T, store kv.Storage, dbName string) session.Session { se, err := session.CreateSession4Test(store) - c.Assert(err, IsNil) - mustExec(c, se, "create database if not exists "+dbName) - mustExec(c, se, "use "+dbName) + require.NoError(t, err) + mustExec(t, se, "create database if not exists "+dbName) + mustExec(t, se, "use "+dbName) return se } -func mustExec(c *C, se session.Session, sql string) { +func mustExec(t *testing.T, se session.Session, sql string) { _, err := se.Execute(context.Background(), sql) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testPrepareSerialSuite) TestConstPropAndPPDWithCache(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestConstPropAndPPDWithCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1341,22 +1209,17 @@ func (s *testPrepareSerialSuite) TestConstPropAndPPDWithCache(c *C) { tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestPlanCacheUnionScan(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPlanCacheUnionScan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) pb := &dto.Metric{} metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() @@ -1373,97 +1236,90 @@ func (s *testPlanSerialSuite) TestPlanCacheUnionScan(c *C) { tk.MustExec("begin") tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt := pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(1)) + require.Equal(t, float64(1), cnt) tk.MustExec("insert into t1 values(1)") // Cached plan is invalid now, it is not chosen and removed. tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows( "1", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(1)) + require.Equal(t, float64(1), cnt) tk.MustExec("insert into t2 values(1)") // Cached plan is chosen, modification on t2 does not impact plan of t1. tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows( "1", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(2)) + require.Equal(t, float64(2), cnt) tk.MustExec("rollback") // Though cached plan contains UnionScan, it does not impact correctness, so it is reused. tk.MustQuery("execute stmt1 using @p0").Check(testkit.Rows()) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(3)) + require.Equal(t, float64(3), cnt) tk.MustExec("prepare stmt2 from 'select * from t1 left join t2 on true where t1.a > ?'") tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) tk.MustExec("begin") tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(4)) + require.Equal(t, float64(4), cnt) tk.MustExec("insert into t1 values(1)") // Cached plan is invalid now, it is not chosen and removed. tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( "1 ", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(4)) + require.Equal(t, float64(4), cnt) tk.MustExec("insert into t2 values(1)") // Cached plan is invalid now, it is not chosen and removed. tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( "1 1", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(4)) + require.Equal(t, float64(4), cnt) // Cached plan is reused. tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows( "1 1", )) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(5)) + require.Equal(t, float64(5), cnt) tk.MustExec("rollback") // Though cached plan contains UnionScan, it does not impact correctness, so it is reused. tk.MustQuery("execute stmt2 using @p0").Check(testkit.Rows()) err = counter.Write(pb) - c.Assert(err, IsNil) + require.NoError(t, err) cnt = pb.GetCounter().GetValue() - c.Check(cnt, Equals, float64(6)) + require.Equal(t, float64(6), cnt) } -func (s *testPlanSerialSuite) TestPlanCacheHitInfo(c *C) { - c.Skip("unstable, skip it and fix it before 20210705") - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPlanCacheHitInfo(t *testing.T) { + t.Skip("unstable, skip it and fix it before 20210705") + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1489,23 +1345,17 @@ func (s *testPlanSerialSuite) TestPlanCacheHitInfo(c *C) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) } -func (s *testPlanSerialSuite) TestIssue29303(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29303(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`set tidb_enable_clustered_index=on`) tk.MustExec(`use test`) @@ -1523,23 +1373,17 @@ func (s *testPlanSerialSuite) TestIssue29303(c *C) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) } -func (s *testPlanSerialSuite) TestIssue28942(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue28942(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists IDT_MULTI15853STROBJSTROBJ`) @@ -1557,24 +1401,17 @@ func (s *testPlanSerialSuite) TestIssue28942(c *C) { tk.MustQuery(`execute stmt using @a,@b`).Check(testkit.Rows()) // empty result } -func (s *testPlanSerialSuite) TestPlanCacheUnsignedHandleOverflow(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPlanCacheUnsignedHandleOverflow(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1591,23 +1428,17 @@ func (s *testPlanSerialSuite) TestPlanCacheUnsignedHandleOverflow(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } -func (s *testPlanSerialSuite) TestIssue28254(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue28254(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists PK_GCOL_STORED9816") @@ -1624,23 +1455,17 @@ func (s *testPlanSerialSuite) TestIssue28254(c *C) { tk.MustQuery("execute stmt using @a").Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestIssue29486(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29486(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists UK_MULTI_COL_11691`) @@ -1661,23 +1486,17 @@ func (s *testPlanSerialSuite) TestIssue29486(c *C) { tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows("126", "126")) } -func (s *testPlanSerialSuite) TestIssue28867(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue28867(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -1710,19 +1529,85 @@ func (s *testPlanSerialSuite) TestIssue28867(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestIssue29565(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestParamMarker4FastPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + + // test handle + tk.MustExec(`use test`) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(pk int primary key)") + tk.MustExec("insert into t values(1)") + tk.MustExec(`prepare stmt from 'select * from t where pk = ?'`) + tk.MustExec(`set @a0=1.1, @a1='1.1', @a2=1, @a3=1.0, @a4='1.0'`) + + tk.MustQuery(`execute stmt using @a2`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a2`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + tk.MustQuery(`execute stmt using @a3`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a3`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a0`).Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + tk.MustQuery(`execute stmt using @a4`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a4`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a1`).Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + // test indexValues + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(pk int, unique index idx(pk))") + tk.MustExec("insert into t values(1)") + tk.MustExec(`prepare stmt from 'select * from t where pk = ?'`) + tk.MustExec(`set @a0=1.1, @a1='1.1', @a2=1, @a3=1.0, @a4='1.0'`) + + tk.MustQuery(`execute stmt using @a2`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a2`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + + tk.MustQuery(`execute stmt using @a3`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a3`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a0`).Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + tk.MustQuery(`execute stmt using @a4`).Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a4`).Check(testkit.Rows("1")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustQuery(`execute stmt using @a1`).Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + // test _tidb_rowid + tk.MustExec(`use test`) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int, b int);") + tk.MustExec("insert t values (1, 7), (1, 8), (1, 9);") + tk.MustExec(`prepare stmt from 'select * from t where _tidb_rowid = ?'`) + tk.MustExec(`set @a=2`) + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1 8")) + tk.MustExec(`set @a=1`) + tk.MustQuery("execute stmt using @a;").Check(testkit.Rows("1 7")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) +} + +func TestIssue29565(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + orgEnable := core.PreparedPlanCacheEnabled() + defer core.SetPreparedPlanCache(orgEnable) + core.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists PK_SIGNED_10094`) @@ -1736,19 +1621,13 @@ func (s *testPlanSerialSuite) TestIssue29565(c *C) { tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestIssue28828(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - store.Close() - }() +func TestIssue28828(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.MustExec("CREATE TABLE t (" + @@ -1782,22 +1661,17 @@ func (s *testPlanSerialSuite) TestIssue28828(c *C) { tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) } -func (s *testPlanSerialSuite) TestIssue28920(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue28920(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists UK_GCOL_VIRTUAL_18928`) @@ -1817,26 +1691,19 @@ func (s *testPlanSerialSuite) TestIssue28920(c *C) { tk.MustQuery(`execute stmt using @a, @b`).Check(testkit.Rows("-5175976006730879891 8 屘厒镇览錻碛斵大擔觏譨頙硺箄魨搝珄鋧扭趖 ")) } -func (s *testPlanSerialSuite) TestIssue18066(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue18066(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) - tk.GetConnectionID() - c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + tk.RefreshConnectionID() + require.True(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1855,16 +1722,10 @@ func (s *testPlanSerialSuite) TestIssue18066(c *C) { testkit.Rows("3 1 0")) } -func (s *testPrepareSuite) TestPrepareForGroupByMultiItems(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() +func TestPrepareForGroupByMultiItems(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1878,8 +1739,7 @@ func (s *testPrepareSuite) TestPrepareForGroupByMultiItems(c *C) { tk.MustQuery(`execute stmt using @a, @b, @a, @b`).Check(testkit.Rows("1 4 -1", "1 2 1", "4 4 3")) tk.MustExec("set @c=10") - err = tk.ExecToErr("execute stmt using @a, @c, @a, @c") - c.Assert(err.Error(), Equals, "Unknown column '10' in 'group statement'") + require.EqualError(t, tk.ExecToErr("execute stmt using @a, @c, @a, @c"), "Unknown column '10' in 'group statement'") tk.MustExec("set @v1=1.0") tk.MustExec("set @v2=3.0") @@ -1887,16 +1747,10 @@ func (s *testPrepareSuite) TestPrepareForGroupByMultiItems(c *C) { tk.MustQuery(`execute stmt2 using @v1, @v2`).Check(testkit.Rows("10")) } -func (s *testPrepareSuite) TestInvisibleIndex(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - }() +func TestInvisibleIndexPrepare(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1906,39 +1760,33 @@ func (s *testPrepareSuite) TestInvisibleIndex(c *C) { tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) - c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 1) + require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) tk.MustExec("alter table t alter index idx_a invisible") tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) - c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 0) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 0) tk.MustExec("alter table t alter index idx_a visible") tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) tk.MustQuery("execute stmt1").Check(testkit.Rows("1")) - c.Assert(len(tk.Se.GetSessionVars().StmtCtx.IndexNames), Equals, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.IndexNames[0], Equals, "t:idx_a") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.IndexNames, 1) + require.Equal(t, "t:idx_a", tk.Session().GetSessionVars().StmtCtx.IndexNames[0]) } // Test for issue https://github.com/pingcap/tidb/issues/22167 -func (s *testPrepareSerialSuite) TestPrepareCacheWithJoinTable(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPrepareCacheWithJoinTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists ta, tb") @@ -1951,23 +1799,17 @@ func (s *testPrepareSerialSuite) TestPrepareCacheWithJoinTable(c *C) { tk.MustQuery("execute stmt using @b").Check(testkit.Rows("a ")) } -func (s *testPlanSerialSuite) TestPlanCacheSnapshot(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPlanCacheSnapshot(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1991,9 +1833,9 @@ func (s *testPlanSerialSuite) TestPlanCacheSnapshot(c *C) { // Record the current tso. tk.MustExec("begin") - tso := tk.Se.GetSessionVars().TxnCtx.StartTS + tso := tk.Session().GetSessionVars().TxnCtx.StartTS tk.MustExec("rollback") - c.Assert(tso > 0, IsTrue) + require.True(t, tso > 0) // Insert one more row with id = 1. tk.MustExec("insert into t values (1)") @@ -2003,22 +1845,17 @@ func (s *testPlanSerialSuite) TestPlanCacheSnapshot(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestPlanCachePointGetAndTableDual(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestPlanCachePointGetAndTableDual(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t0, t1, t2, t3, t4") @@ -2088,22 +1925,17 @@ func (s *testPlanSerialSuite) TestPlanCachePointGetAndTableDual(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPrepareSuite) TestIssue26873(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue26873(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2117,22 +1949,17 @@ func (s *testPrepareSuite) TestIssue26873(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPrepareSuite) TestIssue29511(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29511(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2144,22 +1971,17 @@ func (s *testPrepareSuite) TestIssue29511(c *C) { tk.MustQuery("execute stmt using @a,@b;").Check(testkit.Rows("-3865356285544170443")) } -func (s *testPlanSerialSuite) TestIssue23671(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue23671(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - c.Assert(store.Close(), IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -2174,23 +1996,17 @@ func (s *testPlanSerialSuite) TestIssue23671(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPrepareSerialSuite) TestIssue29296(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29296(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec(`use test`) tk.MustExec(`drop table if exists UK_MU14722`) @@ -2213,23 +2029,17 @@ func (s *testPrepareSerialSuite) TestIssue29296(c *C) { tk.MustQuery(`execute stmt using @a,@b,@c,@d`).Check(testkit.Rows(`偧孇鱓鼂瘠钻篝醗時鷷聽箌磇砀玸眞扦鸇祈灇 127 7902-03-05 08:54:04 -1094128660`)) } -func (s *testPrepareSerialSuite) TestIssue28246(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue28246(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists PK_AUTO_RANDOM9111;") @@ -2247,23 +2057,17 @@ func (s *testPrepareSerialSuite) TestIssue28246(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPrepareSerialSuite) TestIssue29805(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29805(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("set tidb_enable_clustered_index=on;") @@ -2282,23 +2086,17 @@ func (s *testPrepareSerialSuite) TestIssue29805(c *C) { tk.MustQuery("select/*+ hash_agg() */ count(distinct col1) from PK_TCOLLATION10197 where col1 > '龺';").Check(testkit.Rows("0")) } -func (s *testPrepareSerialSuite) TestIssue29993(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue29993(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") @@ -2359,23 +2157,17 @@ func (s *testPrepareSerialSuite) TestIssue29993(c *C) { tk.MustQuery("execute stmt using @z").Check(testkit.Rows()) } -func (s *testPrepareSerialSuite) TestIssue30100(c *C) { - defer testleak.AfterTest(c)() - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) +func TestIssue30100(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) tk.MustExec("use test") tk.MustExec("drop table if exists t;") @@ -2390,31 +2182,24 @@ func (s *testPrepareSerialSuite) TestIssue30100(c *C) { tk.MustExec("set @a=0;") tk.MustQuery("execute stmt using @a").Check(testkit.Rows()) tk.MustQuery("execute stmt using @a").Check(testkit.Rows()) - // If the plan contains the tableDual, it can not be cached. - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } -func (s *testPlanSerialSuite) TestPartitionTable(c *C) { +func TestPartitionTable(t *testing.T) { if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") + t.Skip("exhaustive types test, skip race test") } - // enable plan cache - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) // enable partition table dynamic mode tk.MustExec("create database test_plan_cache") @@ -2536,27 +2321,21 @@ func (s *testPlanSerialSuite) TestPartitionTable(c *C) { } } -func (s *testPlanSerialSuite) TestPartitionWithVariedDatasources(c *C) { +func TestPartitionWithVariedDataSources(t *testing.T) { if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") + t.Skip("exhaustive types test, skip race test") } - // enable plan cache - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - tk := testkit.NewTestKit(c, store) + store, clean := testkit.CreateMockStore(t) + defer clean() orgEnable := core.PreparedPlanCacheEnabled() - defer func() { - dom.Close() - err = store.Close() - c.Assert(err, IsNil) - core.SetPreparedPlanCache(orgEnable) - }() + defer core.SetPreparedPlanCache(orgEnable) core.SetPreparedPlanCache(true) - tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) // enable partition table dynamic mode tk.MustExec("create database test_plan_cache2") @@ -2695,3 +2474,69 @@ func (s *testPlanSerialSuite) TestPartitionWithVariedDatasources(c *C) { } } } + +func TestCachedTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + orgEnable := core.PreparedPlanCacheEnabled() + defer core.SetPreparedPlanCache(orgEnable) + core.SetPreparedPlanCache(true) + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + require.NoError(t, err) + tk := testkit.NewTestKitWithSession(t, store, se) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + + tk.MustExec("create table t (a int, b int, index i_b(b))") + tk.MustExec("insert into t values (1, 1), (2, 2)") + tk.MustExec("alter table t cache") + + tk.MustExec("prepare tableScan from 'select * from t where a>=?'") + tk.MustExec("prepare indexScan from 'select b from t use index(i_b) where b>?'") + tk.MustExec("prepare indexLookup from 'select a from t use index(i_b) where b>? and b mysql.MaxKeyParts { return infoschema.ErrTooManyKeyParts.GenWithStackByArgs(mysql.MaxKeyParts) } + for _, idxSpec := range IndexPartSpecifications { + // -1 => unspecified/full, > 0 OK, 0 => error + if idxSpec.Expr == nil && idxSpec.Length == 0 { + return ErrKeyPart0.GenWithStackByArgs(idxSpec.Column.Name.O) + } + } return checkDuplicateColumnName(IndexPartSpecifications) } diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index ff25091b4eeb8..d5654a4ac7645 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -15,15 +15,12 @@ package core_test import ( - "context" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" @@ -32,54 +29,25 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/testleak" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testValidatorSuite{}) - -type testValidatorSuite struct { - store kv.Storage - dom *domain.Domain - se session.Session - ctx sessionctx.Context - is infoschema.InfoSchema -} - -func (s *testValidatorSuite) SetUpTest(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) - - s.se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - - s.ctx = s.se.(sessionctx.Context) - - s.is = infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable()}) -} - -func (s *testValidatorSuite) TearDownTest(c *C) { - s.dom.Close() - err := s.store.Close() - c.Assert(err, IsNil) - testleak.AfterTest(c)() -} - -func (s *testValidatorSuite) runSQL(c *C, sql string, inPrepare bool, terr error) { - stmts, err1 := session.Parse(s.ctx, sql) - c.Assert(err1, IsNil, Commentf("sql: %s", sql)) - c.Assert(stmts, HasLen, 1) +func runSQL(t *testing.T, ctx sessionctx.Context, is infoschema.InfoSchema, sql string, inPrepare bool, terr error) { + stmts, err := session.Parse(ctx, sql) + require.NoErrorf(t, err, "sql: %s", sql) + require.Len(t, stmts, 1) stmt := stmts[0] var opts []core.PreprocessOpt if inPrepare { opts = append(opts, core.InPrepare) } - err := core.Preprocess(s.ctx, stmt, append(opts, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is}))...) - c.Assert(terror.ErrorEqual(err, terr), IsTrue, Commentf("sql: %s, err:%v", sql, err)) + err = core.Preprocess(ctx, stmt, append(opts, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: is}))...) + require.Truef(t, terror.ErrorEqual(err, terr), "sql: %s, err:%v", sql, err) } -func (s *testValidatorSuite) TestValidator(c *C) { +func TestValidator(t *testing.T) { tests := []struct { sql string inPrepare bool @@ -304,63 +272,72 @@ func (s *testValidatorSuite) TestValidator(c *C) { {"select * from t tablesample system() repeatable (10);", false, expression.ErrInvalidTableSample}, } - _, err := s.se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable()}) for _, tt := range tests { - s.runSQL(c, tt.sql, tt.inPrepare, tt.err) + runSQL(t, tk.Session(), is, tt.sql, tt.inPrepare, tt.err) } } -func (s *testValidatorSuite) TestForeignKey(c *C) { - _, err := s.se.Execute(context.Background(), "create table test.t1(a int, b int, c int)") - c.Assert(err, IsNil) - - _, err = s.se.Execute(context.Background(), "create table test.t2(d int)") - c.Assert(err, IsNil) - - _, err = s.se.Execute(context.Background(), "create database test2") - c.Assert(err, IsNil) - - _, err = s.se.Execute(context.Background(), "create table test2.t(e int)") - c.Assert(err, IsNil) - - s.is = s.dom.InfoSchema() - - s.runSQL(c, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (a) REFERENCES t2 (d)", false, nil) - - _, err = s.se.Execute(context.Background(), "use test") - c.Assert(err, IsNil) - - s.runSQL(c, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (b) REFERENCES t2 (d)", false, nil) +func TestForeignKey(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create table test.t1(a int, b int, c int)") + tk.MustExec("create table test.t2(d int)") + tk.MustExec("create database test2") + tk.MustExec("create table test2.t(e int)") + + is := dom.InfoSchema() + runSQL(t, tk.Session(), is, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (a) REFERENCES t2 (d)", false, nil) + + tk.MustExec("use test") + runSQL(t, tk.Session(), is, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (b) REFERENCES t2 (d)", false, nil) + runSQL(t, tk.Session(), is, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (c) REFERENCES test2.t (e)", false, nil) +} - s.runSQL(c, "ALTER TABLE test.t1 ADD CONSTRAINT fk FOREIGN KEY (c) REFERENCES test2.t (e)", false, nil) +func TestDropGlobalTempTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table tb(id int);") + tk.MustExec("create global temporary table temp(id int) on commit delete rows;") + tk.MustExec("create global temporary table temp1(id int) on commit delete rows;") + tk.MustExec("create temporary table ltemp1(id int);") + tk.MustExec("create database test2") + tk.MustExec("create global temporary table test2.temp2(id int) on commit delete rows;") + + is := tk.Session().GetInfoSchema().(infoschema.InfoSchema) + runSQL(t, tk.Session(), is, "drop global temporary table tb;", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table temp", false, nil) + runSQL(t, tk.Session(), is, "drop global temporary table test.tb;", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table test.temp1", false, nil) + runSQL(t, tk.Session(), is, "drop global temporary table ltemp1", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table test.ltemp1", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table temp, temp1", false, nil) + runSQL(t, tk.Session(), is, "drop global temporary table temp, tb", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table temp, ltemp1", false, core.ErrDropTableOnTemporaryTable) + runSQL(t, tk.Session(), is, "drop global temporary table test2.temp2, temp1", false, nil) } -func (s *testValidatorSuite) TestDropGlobalTempTable(c *C) { - ctx := context.Background() - execSQLList := []string{ - "use test", - "create table tb(id int);", - "create global temporary table temp(id int) on commit delete rows;", - "create global temporary table temp1(id int) on commit delete rows;", - "create temporary table ltemp1(id int);", - "create database test2", - "create global temporary table test2.temp2(id int) on commit delete rows;", - } - for _, execSQL := range execSQLList { - _, err := s.se.Execute(ctx, execSQL) - c.Assert(err, IsNil) - } - s.is = s.se.GetInfoSchema().(infoschema.InfoSchema) - s.runSQL(c, "drop global temporary table tb;", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table temp", false, nil) - s.runSQL(c, "drop global temporary table test.tb;", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table test.temp1", false, nil) - s.runSQL(c, "drop global temporary table ltemp1", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table test.ltemp1", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table temp, temp1", false, nil) - s.runSQL(c, "drop global temporary table temp, tb", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table temp, ltemp1", false, core.ErrDropTableOnTemporaryTable) - s.runSQL(c, "drop global temporary table test2.temp2, temp1", false, nil) +func TestErrKeyPart0(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database TestErrKeyPart") + tk.MustExec("use TestErrKeyPart") + err := tk.ExecToErr("CREATE TABLE `tbl11`(`a` INT(11) NOT NULL, `b` INT(11), PRIMARY KEY (`a`(0))) CHARSET UTF8MB4 COLLATE UTF8MB4_BIN") + require.EqualError(t, err, "[planner:1391]Key part 'a' length cannot be 0") + err = tk.ExecToErr("create table t (a int, b varchar(255), key (b(0)))") + require.EqualError(t, err, "[planner:1391]Key part 'b' length cannot be 0") + err = tk.ExecToErr("create table t (a int, b varchar(255))") + require.NoError(t, err) + err = tk.ExecToErr("alter table t add index (b(0))") + require.EqualError(t, err, "[planner:1391]Key part 'b' length cannot be 0") } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 17dcb87965e77..d79d7aa7760a3 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -180,10 +180,10 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, return nil } -func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (new []*util.ByItems, +func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (byItems []*util.ByItems, parentUsedCols []*expression.Column) { prunedByItems := make([]*util.ByItems, 0) - new = make([]*util.ByItems, 0, len(old)) + byItems = make([]*util.ByItems, 0, len(old)) seen := make(map[string]struct{}, len(old)) for _, byItem := range old { pruned := true @@ -196,14 +196,14 @@ func pruneByItems(p LogicalPlan, old []*util.ByItems, opt *logicalOptimizeOp) (n } else if len(cols) == 0 { if !expression.IsRuntimeConstExpr(byItem.Expr) { pruned = false - new = append(new, byItem) + byItems = append(byItems, byItem) } } else if byItem.Expr.GetType().Tp == mysql.TypeNull { // do nothing, should be filtered } else { pruned = false parentUsedCols = append(parentUsedCols, cols...) - new = append(new, byItem) + byItems = append(byItems, byItem) } if pruned { prunedByItems = append(prunedByItems, byItem) @@ -310,6 +310,12 @@ func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *log originColumns := ds.Columns for i := len(used) - 1; i >= 0; i-- { if !used[i] && !exprUsed[i] { + // If ds has a shard index, and the column is generated column by `tidb_shard()` + // it can't prune the generated column of shard index + if ds.containExprPrefixUk && + expression.GcColumnExprIsTidbShard(ds.schema.Columns[i].VirtualExpr) { + continue + } prunedColumns = append(prunedColumns, ds.schema.Columns[i]) ds.schema.Columns = append(ds.schema.Columns[:i], ds.schema.Columns[i+1:]...) ds.Columns = append(ds.Columns[:i], ds.Columns[i+1:]...) diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 182597eca6210..e8a80f722f055 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -154,6 +154,21 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo return s.optimize(ctx, p, opt) } } else if proj, ok := innerPlan.(*LogicalProjection); ok { + allConst := true + for _, expr := range proj.Exprs { + if len(expression.ExtractCorColumns(expr)) > 0 || !expression.ExtractColumnSet(expr).IsEmpty() { + allConst = false + break + } + } + if allConst && apply.JoinType == LeftOuterJoin { + // If the projection just references some constant. We cannot directly pull it up when the APPLY is an outer join. + // e.g. select (select 1 from t1 where t1.a=t2.a) from t2; When the t1.a=t2.a is false the join's output is NULL. + // But if we pull the projection upon the APPLY. It will return 1 since the projection is evaluated after the join. + // We disable the decorrelation directly for now. + // TODO: Actually, it can be optimized. We need to first push the projection down to the selection. And then the APPLY can be decorrelated. + goto NoOptimize + } for i, expr := range proj.Exprs { proj.Exprs[i] = expr.Decorrelate(outerPlan.Schema()) } @@ -315,6 +330,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo return s.optimize(ctx, p, opt) } } +NoOptimize: newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { np, err := s.optimize(ctx, child, opt) diff --git a/planner/core/rule_inject_extra_projection_test.go b/planner/core/rule_inject_extra_projection_test.go index 6bbfc544638ff..58b00f0113848 100644 --- a/planner/core/rule_inject_extra_projection_test.go +++ b/planner/core/rule_inject_extra_projection_test.go @@ -15,21 +15,18 @@ package core import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testInjectProjSuite{}) - -type testInjectProjSuite struct { -} - -func (s *testInjectProjSuite) TestWrapCastForAggFuncs(c *C) { +func TestWrapCastForAggFuncs(t *testing.T) { aggNames := []string{ast.AggFuncSum} modes := []aggregation.AggFunctionMode{aggregation.CompleteMode, aggregation.FinalMode, aggregation.Partial1Mode, aggregation.Partial1Mode} @@ -45,7 +42,7 @@ func (s *testInjectProjSuite) TestWrapCastForAggFuncs(c *C) { aggFunc, err := aggregation.NewAggFuncDesc(sctx, name, []expression.Expression{&expression.Constant{Value: types.Datum{}, RetType: types.NewFieldType(retType)}}, hasDistinct) - c.Assert(err, IsNil) + require.NoError(t, err) aggFunc.Mode = mode aggFuncs = append(aggFuncs, aggFunc) } @@ -61,9 +58,9 @@ func (s *testInjectProjSuite) TestWrapCastForAggFuncs(c *C) { wrapCastForAggFuncs(mock.NewContext(), aggFuncs) for i := range aggFuncs { if aggFuncs[i].Mode != aggregation.FinalMode && aggFuncs[i].Mode != aggregation.Partial2Mode { - c.Assert(aggFuncs[i].RetTp.Tp, Equals, aggFuncs[i].Args[0].GetType().Tp) + require.Equal(t, aggFuncs[i].Args[0].GetType().Tp, aggFuncs[i].RetTp.Tp) } else { - c.Assert(aggFuncs[i].Args[0].GetType().Tp, Equals, orgAggFuncs[i].Args[0].GetType().Tp) + require.Equal(t, orgAggFuncs[i].Args[0].GetType().Tp, aggFuncs[i].Args[0].GetType().Tp) } } } diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index 5d674dca9e66a..ed41d55c43520 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -16,8 +16,8 @@ package core import ( "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" @@ -25,20 +25,9 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testJoinReorderDPSuite{}) - -type testJoinReorderDPSuite struct { - ctx sessionctx.Context - statsMap map[int]*property.StatsInfo -} - -func (s *testJoinReorderDPSuite) SetUpTest(c *C) { - s.ctx = MockContext() - s.ctx.GetSessionVars().PlanID = -1 -} - type mockLogicalJoin struct { logicalSchemaProducer involvedNodeSet int @@ -57,31 +46,27 @@ func (mj *mockLogicalJoin) recursiveDeriveStats(_ [][]*expression.Column) (*prop return mj.statsMap[mj.involvedNodeSet], nil } -func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { - retJoin := mockLogicalJoin{}.init(s.ctx) - retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) - retJoin.statsMap = s.statsMap - if mj, ok := lChild.(*mockLogicalJoin); ok { - retJoin.involvedNodeSet = mj.involvedNodeSet - } else { - retJoin.involvedNodeSet = 1 << uint(lChild.ID()) - } - if mj, ok := rChild.(*mockLogicalJoin); ok { - retJoin.involvedNodeSet |= mj.involvedNodeSet - } else { - retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) - } - retJoin.SetChildren(lChild, rChild) - return retJoin -} - -func (s *testJoinReorderDPSuite) mockStatsInfo(state int, count float64) { - s.statsMap[state] = &property.StatsInfo{ - RowCount: count, +func newMockJoin(ctx sessionctx.Context, statsMap map[int]*property.StatsInfo) func(lChild, rChild LogicalPlan, _ []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { + return func(lChild, rChild LogicalPlan, _ []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { + retJoin := mockLogicalJoin{}.init(ctx) + retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) + retJoin.statsMap = statsMap + if mj, ok := lChild.(*mockLogicalJoin); ok { + retJoin.involvedNodeSet = mj.involvedNodeSet + } else { + retJoin.involvedNodeSet = 1 << uint(lChild.ID()) + } + if mj, ok := rChild.(*mockLogicalJoin); ok { + retJoin.involvedNodeSet |= mj.involvedNodeSet + } else { + retJoin.involvedNodeSet |= 1 << uint(rChild.ID()) + } + retJoin.SetChildren(lChild, rChild) + return retJoin } } -func (s *testJoinReorderDPSuite) makeStatsMapForTPCHQ5() { +func makeStatsMapForTPCHQ5() map[int]*property.StatsInfo { // Labeled as lineitem -> 0, orders -> 1, customer -> 2, supplier 3, nation 4, region 5 // This graph can be shown as following: // +---------------+ +---------------+ @@ -112,48 +97,48 @@ func (s *testJoinReorderDPSuite) makeStatsMapForTPCHQ5() { // | region | // | | // +---------------+ - s.statsMap = make(map[int]*property.StatsInfo) - s.mockStatsInfo(3, 9103367) - s.mockStatsInfo(6, 2275919) - s.mockStatsInfo(7, 9103367) - s.mockStatsInfo(9, 59986052) - s.mockStatsInfo(11, 9103367) - s.mockStatsInfo(12, 5999974575) - s.mockStatsInfo(13, 59999974575) - s.mockStatsInfo(14, 9103543072) - s.mockStatsInfo(15, 99103543072) - s.mockStatsInfo(20, 1500000) - s.mockStatsInfo(22, 2275919) - s.mockStatsInfo(23, 7982159) - s.mockStatsInfo(24, 100000) - s.mockStatsInfo(25, 59986052) - s.mockStatsInfo(27, 9103367) - s.mockStatsInfo(28, 5999974575) - s.mockStatsInfo(29, 59999974575) - s.mockStatsInfo(30, 59999974575) - s.mockStatsInfo(31, 59999974575) - s.mockStatsInfo(48, 5) - s.mockStatsInfo(52, 299838) - s.mockStatsInfo(54, 454183) - s.mockStatsInfo(55, 1815222) - s.mockStatsInfo(56, 20042) - s.mockStatsInfo(57, 12022687) - s.mockStatsInfo(59, 1823514) - s.mockStatsInfo(60, 1201884359) - s.mockStatsInfo(61, 12001884359) - s.mockStatsInfo(62, 12001884359) - s.mockStatsInfo(63, 72985) - + statsMap := make(map[int]*property.StatsInfo) + statsMap[3] = &property.StatsInfo{RowCount: 9103367} + statsMap[6] = &property.StatsInfo{RowCount: 2275919} + statsMap[7] = &property.StatsInfo{RowCount: 9103367} + statsMap[9] = &property.StatsInfo{RowCount: 59986052} + statsMap[11] = &property.StatsInfo{RowCount: 9103367} + statsMap[12] = &property.StatsInfo{RowCount: 5999974575} + statsMap[13] = &property.StatsInfo{RowCount: 59999974575} + statsMap[14] = &property.StatsInfo{RowCount: 9103543072} + statsMap[15] = &property.StatsInfo{RowCount: 99103543072} + statsMap[20] = &property.StatsInfo{RowCount: 1500000} + statsMap[22] = &property.StatsInfo{RowCount: 2275919} + statsMap[23] = &property.StatsInfo{RowCount: 7982159} + statsMap[24] = &property.StatsInfo{RowCount: 100000} + statsMap[25] = &property.StatsInfo{RowCount: 59986052} + statsMap[27] = &property.StatsInfo{RowCount: 9103367} + statsMap[28] = &property.StatsInfo{RowCount: 5999974575} + statsMap[29] = &property.StatsInfo{RowCount: 59999974575} + statsMap[30] = &property.StatsInfo{RowCount: 59999974575} + statsMap[31] = &property.StatsInfo{RowCount: 59999974575} + statsMap[48] = &property.StatsInfo{RowCount: 5} + statsMap[52] = &property.StatsInfo{RowCount: 299838} + statsMap[54] = &property.StatsInfo{RowCount: 454183} + statsMap[55] = &property.StatsInfo{RowCount: 1815222} + statsMap[56] = &property.StatsInfo{RowCount: 20042} + statsMap[57] = &property.StatsInfo{RowCount: 12022687} + statsMap[59] = &property.StatsInfo{RowCount: 1823514} + statsMap[60] = &property.StatsInfo{RowCount: 1201884359} + statsMap[61] = &property.StatsInfo{RowCount: 12001884359} + statsMap[62] = &property.StatsInfo{RowCount: 12001884359} + statsMap[63] = &property.StatsInfo{RowCount: 72985} + return statsMap } -func (s *testJoinReorderDPSuite) newDataSource(name string, count int) LogicalPlan { - ds := DataSource{}.Init(s.ctx, 0) +func newDataSource(ctx sessionctx.Context, name string, count int) LogicalPlan { + ds := DataSource{}.Init(ctx, 0) tan := model.NewCIStr(name) ds.TableAsName = &tan ds.schema = expression.NewSchema() - s.ctx.GetSessionVars().PlanColumnID++ + ctx.GetSessionVars().PlanColumnID++ ds.schema.Append(&expression.Column{ - UniqueID: s.ctx.GetSessionVars().PlanColumnID, + UniqueID: ctx.GetSessionVars().PlanColumnID, RetType: types.NewFieldType(mysql.TypeLonglong), }) ds.stats = &property.StatsInfo{ @@ -162,57 +147,70 @@ func (s *testJoinReorderDPSuite) newDataSource(name string, count int) LogicalPl return ds } -func (s *testJoinReorderDPSuite) planToString(plan LogicalPlan) string { +func planToString(plan LogicalPlan) string { switch x := plan.(type) { case *mockLogicalJoin: - return fmt.Sprintf("MockJoin{%v, %v}", s.planToString(x.children[0]), s.planToString(x.children[1])) + return fmt.Sprintf("MockJoin{%v, %v}", planToString(x.children[0]), planToString(x.children[1])) case *DataSource: return x.TableAsName.L } return "" } -func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { - s.makeStatsMapForTPCHQ5() +func TestDPReorderTPCHQ5(t *testing.T) { + statsMap := makeStatsMapForTPCHQ5() + + ctx := MockContext() + ctx.GetSessionVars().PlanID = -1 joinGroups := make([]LogicalPlan, 0, 6) - joinGroups = append(joinGroups, s.newDataSource("lineitem", 59986052)) - joinGroups = append(joinGroups, s.newDataSource("orders", 15000000)) - joinGroups = append(joinGroups, s.newDataSource("customer", 1500000)) - joinGroups = append(joinGroups, s.newDataSource("supplier", 100000)) - joinGroups = append(joinGroups, s.newDataSource("nation", 25)) - joinGroups = append(joinGroups, s.newDataSource("region", 5)) + joinGroups = append(joinGroups, newDataSource(ctx, "lineitem", 59986052)) + joinGroups = append(joinGroups, newDataSource(ctx, "orders", 15000000)) + joinGroups = append(joinGroups, newDataSource(ctx, "customer", 1500000)) + joinGroups = append(joinGroups, newDataSource(ctx, "supplier", 100000)) + joinGroups = append(joinGroups, newDataSource(ctx, "nation", 25)) + joinGroups = append(joinGroups, newDataSource(ctx, "region", 5)) + var eqConds []expression.Expression - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[1].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[1].Schema().Columns[0], joinGroups[2].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[3].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[3].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) - eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[1].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[1].Schema().Columns[0], joinGroups[2].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[3].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[3].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[2].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) + eqConds = append(eqConds, expression.NewFunctionInternal(ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) solver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ - ctx: s.ctx, + ctx: ctx, }, - newJoin: s.newMockJoin, + newJoin: newMockJoin(ctx, statsMap), } result, err := solver.solve(joinGroups, eqConds, nil) - c.Assert(err, IsNil) - c.Assert(s.planToString(result), Equals, "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}") + require.NoError(t, err) + + expected := "MockJoin{supplier, MockJoin{lineitem, MockJoin{orders, MockJoin{customer, MockJoin{nation, region}}}}}" + require.Equal(t, expected, planToString(result)) } -func (s *testJoinReorderDPSuite) TestDPReorderAllCartesian(c *C) { +func TestDPReorderAllCartesian(t *testing.T) { + statsMap := makeStatsMapForTPCHQ5() + + ctx := MockContext() + ctx.GetSessionVars().PlanID = -1 + joinGroup := make([]LogicalPlan, 0, 4) - joinGroup = append(joinGroup, s.newDataSource("a", 100)) - joinGroup = append(joinGroup, s.newDataSource("b", 100)) - joinGroup = append(joinGroup, s.newDataSource("c", 100)) - joinGroup = append(joinGroup, s.newDataSource("d", 100)) + joinGroup = append(joinGroup, newDataSource(ctx, "a", 100)) + joinGroup = append(joinGroup, newDataSource(ctx, "b", 100)) + joinGroup = append(joinGroup, newDataSource(ctx, "c", 100)) + joinGroup = append(joinGroup, newDataSource(ctx, "d", 100)) solver := &joinReorderDPSolver{ baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ - ctx: s.ctx, + ctx: ctx, }, - newJoin: s.newMockJoin, + newJoin: newMockJoin(ctx, statsMap), } result, err := solver.solve(joinGroup, nil, nil) - c.Assert(err, IsNil) - c.Assert(s.planToString(result), Equals, "MockJoin{MockJoin{a, b}, MockJoin{c, d}}") + require.NoError(t, err) + + expected := "MockJoin{MockJoin{a, b}, MockJoin{c, d}}" + require.Equal(t, expected, planToString(result)) } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 8f83f6757c51c..0597298fc2cc7 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -930,7 +930,7 @@ func makePartitionByFnCol(sctx sessionctx.Context, columns []*expression.Column, monotonous = getMonotoneMode(raw.FuncName.L) // Check the partitionExpr is in the form: fn(col, ...) // There should be only one column argument, and it should be the first parameter. - if expression.ExtractColumnSet(args).Len() == 1 { + if expression.ExtractColumnSet(args...).Len() == 1 { if col1, ok := args[0].(*expression.Column); ok { col = col1 } diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 2d90c647933e1..9896b844f74f8 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -23,12 +23,25 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/ranger" + "go.uber.org/zap" ) type ppdSolver struct{} +// exprPrefixAdder is the wrapper struct to add tidb_shard(x) = val for `OrigConds` +// `cols` is the index columns for a unique shard index +type exprPrefixAdder struct { + sctx sessionctx.Context + OrigConds []expression.Expression + cols []*expression.Column + lengths []int +} + func (s *ppdSolver) optimize(ctx context.Context, lp LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { _, p := lp.PredicatePushDown(nil, opt) return p, nil @@ -126,6 +139,9 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression, func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { predicates = expression.PropagateConstant(ds.ctx, predicates) predicates = DeleteTrueExprs(ds, predicates) + // Add tidb_shard() prefix to the condtion for shard index in some scenarios + // TODO: remove it to the place building logical plan + predicates = ds.AddPrefix4ShardIndexes(ds.ctx, predicates) ds.allConds = predicates ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified) appendDataSourcePredicatePushDownTraceStep(ds, opt) @@ -726,3 +742,121 @@ func appendAddSelectionTraceStep(p LogicalPlan, child LogicalPlan, sel *LogicalS } opt.appendStepToCurrent(sel.ID(), sel.TP(), reason, action) } + +// AddPrefix4ShardIndexes add expression prefix for shard index. e.g. an index is test.uk(tidb_shard(a), a). +// It transforms the sql "SELECT * FROM test WHERE a = 10" to +// "SELECT * FROM test WHERE tidb_shard(a) = val AND a = 10", val is the value of tidb_shard(10). +// It also transforms the sql "SELECT * FROM test WHERE a IN (10, 20, 30)" to +// "SELECT * FROM test WHERE tidb_shard(a) = val1 AND a = 10 OR tidb_shard(a) = val2 AND a = 20" +// @param[in] conds the original condtion of this datasource +// @retval - the new condition after adding expression prefix +func (ds *DataSource) AddPrefix4ShardIndexes(sc sessionctx.Context, conds []expression.Expression) []expression.Expression { + if !ds.containExprPrefixUk { + return conds + } + + var err error + newConds := conds + + for _, path := range ds.possibleAccessPaths { + if !path.IsUkShardIndexPath { + continue + } + newConds, err = ds.addExprPrefixCond(sc, path, newConds) + if err != nil { + logutil.BgLogger().Error("Add tidb_shard expression failed", + zap.Error(err), + zap.Uint64("connection id", sc.GetSessionVars().ConnectionID), + zap.String("database name", ds.DBName.L), + zap.String("table name", ds.tableInfo.Name.L), + zap.String("index name", path.Index.Name.L)) + return conds + } + } + + return newConds +} + +func (ds *DataSource) addExprPrefixCond(sc sessionctx.Context, path *util.AccessPath, + conds []expression.Expression) ([]expression.Expression, error) { + IdxCols, IdxColLens := + expression.IndexInfo2PrefixCols(ds.Columns, ds.schema.Columns, path.Index) + if len(IdxCols) == 0 { + return conds, nil + } + + adder := &exprPrefixAdder{ + sctx: sc, + OrigConds: conds, + cols: IdxCols, + lengths: IdxColLens, + } + + return adder.addExprPrefix4ShardIndex() +} + +// AddExprPrefix4ShardIndex +// if original condition is a LogicOr expression, such as `WHERE a = 1 OR a = 10`, +// call the function AddExprPrefix4DNFCond to add prefix expression tidb_shard(a) = xxx for shard index. +// Otherwise, if the condition is `WHERE a = 1`, `WHERE a = 1 AND b = 10`, `WHERE a IN (1, 2, 3)`......, +// call the function AddExprPrefix4CNFCond to add prefix expression for shard index. +func (adder *exprPrefixAdder) addExprPrefix4ShardIndex() ([]expression.Expression, error) { + if len(adder.OrigConds) == 1 { + if sf, ok := adder.OrigConds[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicOr { + return adder.addExprPrefix4DNFCond(sf) + } + } + return adder.addExprPrefix4CNFCond(adder.OrigConds) +} + +// AddExprPrefix4CNFCond +// add the prefix expression for CNF condition, e.g. `WHERE a = 1`, `WHERE a = 1 AND b = 10`, ...... +// @param[in] conds the original condtion of the datasoure. e.g. `WHERE t1.a = 1 AND t1.b = 10 AND t2.a = 20`. +// if current datasource is `t1`, conds is {t1.a = 1, t1.b = 10}. if current datasource is +// `t2`, conds is {t2.a = 20} +// @return - the new condition after adding expression prefix +func (adder *exprPrefixAdder) addExprPrefix4CNFCond(conds []expression.Expression) ([]expression.Expression, error) { + newCondtionds, err := ranger.AddExpr4EqAndInCondition(adder.sctx, + conds, adder.cols) + + return newCondtionds, err +} + +// AddExprPrefix4DNFCond +// add the prefix expression for DNF condition, e.g. `WHERE a = 1 OR a = 10`, ...... +// The condition returned is `WHERE (tidb_shard(a) = 214 AND a = 1) OR (tidb_shard(a) = 142 AND a = 10)` +// @param[in] condition the original condtion of the datasoure. e.g. `WHERE a = 1 OR a = 10`. +// condtion is `a = 1 OR a = 10` +// @return - the new condition after adding expression prefix. It's still a LogicOr expression. +func (adder *exprPrefixAdder) addExprPrefix4DNFCond(condition *expression.ScalarFunction) ([]expression.Expression, error) { + var err error + dnfItems := expression.FlattenDNFConditions(condition) + newAccessItems := make([]expression.Expression, 0, len(dnfItems)) + + for _, item := range dnfItems { + if sf, ok := item.(*expression.ScalarFunction); ok { + var accesses []expression.Expression + if sf.FuncName.L == ast.LogicAnd { + cnfItems := expression.FlattenCNFConditions(sf) + accesses, err = adder.addExprPrefix4CNFCond(cnfItems) + if err != nil { + return []expression.Expression{condition}, err + } + newAccessItems = append(newAccessItems, expression.ComposeCNFCondition(adder.sctx, accesses...)) + } else if sf.FuncName.L == ast.EQ || sf.FuncName.L == ast.In { + // only add prefix expression for EQ or IN function + accesses, err = adder.addExprPrefix4CNFCond([]expression.Expression{sf}) + if err != nil { + return []expression.Expression{condition}, err + } + newAccessItems = append(newAccessItems, expression.ComposeCNFCondition(adder.sctx, accesses...)) + } else { + newAccessItems = append(newAccessItems, item) + } + } else { + newAccessItems = append(newAccessItems, item) + } + } + + return []expression.Expression{expression.ComposeDNFCondition(adder.sctx, newAccessItems...)}, nil +} diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 7c12e1fff934d..567c1488508d9 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -17,49 +17,32 @@ package core_test import ( "fmt" "math" + "testing" - . "github.com/pingcap/check" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/kvcache" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testRuleReorderResults{}) -var _ = SerialSuites(&testRuleReorderResultsSerial{}) +func TestPlanCache(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() -type testRuleReorderResultsSerial struct { - store kv.Storage - dom *domain.Domain -} - -func (s *testRuleReorderResultsSerial) SetUpTest(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testRuleReorderResultsSerial) TearDownTest(c *C) { - s.dom.Close() - c.Assert(s.store.Close(), IsNil) -} - -func (s *testRuleReorderResultsSerial) TestPlanCache(c *C) { - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { plannercore.SetPreparedPlanCache(orgEnable) }() plannercore.SetPreparedPlanCache(true) - var err error - tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{ + se, err := session.CreateSession4TestWithOpt(store, &session.Opt{ PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), }) - c.Assert(err, IsNil) + require.NoError(t, err) + tk.SetSession(se) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") @@ -73,8 +56,11 @@ func (s *testRuleReorderResultsSerial) TestPlanCache(c *C) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) // plan cache is still working } -func (s *testRuleReorderResultsSerial) TestSQLBinding(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestSQLBinding(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("set tidb_opt_limit_push_down_threshold=0") @@ -96,11 +82,14 @@ func (s *testRuleReorderResultsSerial) TestSQLBinding(c *C) { " └─TableRowIDScan_16(Probe) 1.00 cop[tikv] table:t keep order:false, stats:pseudo")) } -func (s *testRuleReorderResultsSerial) TestClusteredIndex(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestClusteredIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE t (a int,b int,c int, PRIMARY KEY (a,b))") tk.MustQuery("explain format=brief select * from t limit 10").Check(testkit.Rows( @@ -108,100 +97,97 @@ func (s *testRuleReorderResultsSerial) TestClusteredIndex(c *C) { "└─TableReader 10.00 root data:TopN", " └─TopN 10.00 cop[tikv] test.t.a, test.t.b, test.t.c, offset:0, count:10", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo")) - tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff -} - -type testRuleReorderResults struct { - store kv.Storage - dom *domain.Domain - - testData testutil.TestData + tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff } -func (s *testRuleReorderResults) SetUpSuite(c *C) { - var err error - s.store, s.dom, err = newStoreWithBootstrap() - c.Assert(err, IsNil) - - s.testData, err = testutil.LoadTestSuiteData("testdata", "ordered_result_mode_suite") - c.Assert(err, IsNil) -} - -func (s *testRuleReorderResults) TearDownSuite(c *C) { - s.dom.Close() - c.Assert(s.store.Close(), IsNil) - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testRuleReorderResults) runTestData(c *C, tk *testkit.TestKit, name string) { +func runTestData(t *testing.T, tk *testkit.TestKit, name string) { var input []string var output []struct { Plan []string } - s.testData.GetTestCasesByName(name, c, &input, &output) - c.Assert(len(input), Equals, len(output)) + statsSuiteData := plannercore.GetOrderedResultModeSuiteData() + statsSuiteData.GetTestCasesByName(name, t, &input, &output) + require.Equal(t, len(input), len(output)) for i := range input { - s.testData.OnRecord(func() { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain " + input[i]).Rows()) + testdata.OnRecord(func() { + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain " + input[i]).Rows()) }) tk.MustQuery("explain " + input[i]).Check(testkit.Rows(output[i].Plan...)) } } -func (s *testRuleReorderResults) TestOrderedResultMode(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultMode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int primary key, b int, c int, d int, key(b))") - s.runTestData(c, tk, "TestOrderedResultMode") + runTestData(t, tk, "TestOrderedResultMode") } -func (s *testRuleReorderResults) TestOrderedResultModeOnDML(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultModeOnDML(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int primary key, b int, c int, key(b))") - s.runTestData(c, tk, "TestOrderedResultModeOnDML") + runTestData(t, tk, "TestOrderedResultModeOnDML") } -func (s *testRuleReorderResults) TestOrderedResultModeOnSubQuery(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultModeOnSubQuery(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") - s.runTestData(c, tk, "TestOrderedResultModeOnSubQuery") + runTestData(t, tk, "TestOrderedResultModeOnSubQuery") } -func (s *testRuleReorderResults) TestOrderedResultModeOnJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultModeOnJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") - s.runTestData(c, tk, "TestOrderedResultModeOnJoin") + runTestData(t, tk, "TestOrderedResultModeOnJoin") } -func (s *testRuleReorderResults) TestOrderedResultModeOnOtherOperators(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultModeOnOtherOperators(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") tk.MustExec("create table t1 (a int primary key, b int, c int, d int, unique key(b))") tk.MustExec("create table t2 (a int primary key, b int, c int, d int, unique key(b))") - s.runTestData(c, tk, "TestOrderedResultModeOnOtherOperators") + runTestData(t, tk, "TestOrderedResultModeOnOtherOperators") } -func (s *testRuleReorderResults) TestOrderedResultModeOnPartitionTable(c *C) { - tk := testkit.NewTestKit(c, s.store) +func TestOrderedResultModeOnPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec(fmt.Sprintf(`set tidb_partition_prune_mode='%v'`, variable.DefTiDBPartitionPruneMode)) tk.MustExec("set tidb_enable_ordered_result_mode=1") @@ -214,10 +200,14 @@ func (s *testRuleReorderResults) TestOrderedResultModeOnPartitionTable(c *C) { partition p2 values less than (300), partition p3 values less than (400))`) tk.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows("static")) - s.runTestData(c, tk, "TestOrderedResultModeOnPartitionTable") + runTestData(t, tk, "TestOrderedResultModeOnPartitionTable") } -func (s *testRuleReorderResults) TestStableResultSwitch(c *C) { - tk := testkit.NewTestKit(c, s.store) - c.Assert(len(tk.MustQuery("show variables where variable_name like 'tidb_enable_ordered_result_mode'").Rows()), Equals, 1) +func TestStableResultSwitch(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + rows := tk.MustQuery("show variables where variable_name like 'tidb_enable_ordered_result_mode'").Rows() + require.Len(t, rows, 1) } diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index 0be76a053fd6d..103c4107c7f5e 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -25,7 +25,10 @@ import ( "github.com/pingcap/tidb/util/stringutil" ) -var _ ShowPredicateExtractor = &ShowColumnsTableExtractor{} +var ( + _ ShowPredicateExtractor = &ShowColumnsTableExtractor{} + _ ShowPredicateExtractor = &ShowTablesTableExtractor{} +) // ShowPredicateExtractor is used to extract some predicates from `PatternLikeExpr` clause // and push the predicates down to the data retrieving on reading memory table stage when use ShowStmt. @@ -75,7 +78,6 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { return true } return false - } func (e *ShowColumnsTableExtractor) explainInfo() string { @@ -95,3 +97,47 @@ func (e *ShowColumnsTableExtractor) explainInfo() string { } return s } + +// ShowTablesTableExtractor is used to extract some predicates of tables. +type ShowTablesTableExtractor struct { + ShowColumnsTableExtractor +} + +// Extract implements the ShowTablesTableExtractor Extract interface +func (e *ShowTablesTableExtractor) Extract(show *ast.ShowStmt) bool { + if show.Pattern != nil && show.Pattern.Pattern != nil { + pattern := show.Pattern + switch pattern.Pattern.(type) { + case *driver.ValueExpr: + // It is used in `SHOW TABLE in t LIKE `abc``. + ptn := pattern.Pattern.(*driver.ValueExpr).GetString() + patValue, patTypes := stringutil.CompilePattern(ptn, pattern.Escape) + if stringutil.IsExactMatch(patTypes) { + e.Field = strings.ToLower(string(patValue)) + return true + } + // (?i) mean to be case-insensitive. + e.FieldPatterns = "(?i)" + stringutil.CompileLike2Regexp(string(patValue)) + return true + } + } + return false +} + +func (e *ShowTablesTableExtractor) explainInfo() string { + r := new(bytes.Buffer) + if len(e.Field) > 0 { + r.WriteString(fmt.Sprintf("table:[%s], ", e.Field)) + } + + if len(e.FieldPatterns) > 0 { + r.WriteString(fmt.Sprintf("table_pattern:[%s], ", e.FieldPatterns)) + } + + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} diff --git a/planner/core/stats.go b/planner/core/stats.go index 3eb26049e63f4..729ec2cbd559d 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -443,8 +443,7 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * } } - readFromTableCache := stmtCtx.ReadFromTableCache - if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal && !readFromTableCache { + if isPossibleIdxMerge && sessionAndStmtPermission && needConsiderIndexMerge && ds.tableInfo.TempTableType != model.TempTableLocal { err := ds.generateAndPruneIndexMergePath(indexMergeConds, ds.indexMergeHints != nil) if err != nil { return nil, err @@ -458,8 +457,6 @@ func (ds *DataSource) DeriveStats(childStats []*property.StatsInfo, selfSchema * msg = "Got no_index_merge hint or tidb_enable_index_merge is off." } else if ds.tableInfo.TempTableType == model.TempTableLocal { msg = "Cannot use IndexMerge on temporary table." - } else if readFromTableCache { - msg = "Cannot use IndexMerge on TableCache." } msg = fmt.Sprintf("IndexMerge is inapplicable or disabled. %s", msg) stmtCtx.AppendWarning(errors.Errorf(msg)) diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index 0485083f99d3b..2959aae593e08 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -16,44 +16,22 @@ package core_test import ( "context" + "fmt" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/util/hint" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testutil" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testStatsSuite{}) - -type testStatsSuite struct { - *parser.Parser - testData testutil.TestData -} - -func (s *testStatsSuite) SetUpSuite(c *C) { - s.Parser = parser.New() - s.Parser.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) - - var err error - s.testData, err = testutil.LoadTestSuiteData("testdata", "stats_suite") - c.Assert(err, IsNil) -} - -func (s *testStatsSuite) TearDownSuite(c *C) { - c.Assert(s.testData.GenerateOutputIfNeeded(), IsNil) -} - -func (s *testStatsSuite) TestGroupNDVs(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestGroupNDVs(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int not null, key(a,b))") @@ -64,29 +42,31 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { tk.MustExec("analyze table t2") ctx := context.Background() + p := parser.New() var input []string var output []struct { SQL string AggInput string JoinInput string } - s.testData.GetTestCases(c, &input, &output) + statsSuiteData := core.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - comment := Commentf("case:%v sql: %s", i, tt) - stmt, err := s.ParseOneStmt(tt, "", "") - c.Assert(err, IsNil, comment) + comment := fmt.Sprintf("case:%v sql: %s", i, tt) + stmt, err := p.ParseOneStmt(tt, "", "") + require.NoError(t, err, comment) ret := &core.PreprocessorReturn{} - err = core.Preprocess(tk.Se, stmt, core.WithPreprocessorReturn(ret)) - c.Assert(err, IsNil) - tk.Se.GetSessionVars().PlanColumnID = 0 - builder, _ := core.NewPlanBuilder().Init(tk.Se, ret.InfoSchema, &hint.BlockHintProcessor{}) + err = core.Preprocess(tk.Session(), stmt, core.WithPreprocessorReturn(ret)) + require.NoError(t, err) + tk.Session().GetSessionVars().PlanColumnID = 0 + builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) p, err = core.LogicalOptimize(ctx, builder.GetOptFlag(), p.(core.LogicalPlan)) - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) lp := p.(core.LogicalPlan) _, err = core.RecursiveDeriveStats4Test(lp) - c.Assert(err, IsNil, comment) + require.NoError(t, err, comment) var agg *core.LogicalAggregation var join *core.LogicalJoin stack := make([]core.LogicalPlan, 0, 2) @@ -130,24 +110,20 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { r := core.GetStats4Test(join.Children()[1]) joinInput = property.ToString(l.GroupNDVs) + ";" + property.ToString(r.GroupNDVs) } - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt output[i].AggInput = aggInput output[i].JoinInput = joinInput }) - c.Assert(aggInput, Equals, output[i].AggInput, comment) - c.Assert(joinInput, Equals, output[i].JoinInput, comment) + require.Equal(t, output[i].AggInput, aggInput, comment) + require.Equal(t, output[i].JoinInput, joinInput, comment) } } -func (s *testStatsSuite) TestNDVGroupCols(c *C) { - store, dom, err := newStoreWithBootstrap() - c.Assert(err, IsNil) - defer func() { - dom.Close() - store.Close() - }() - tk := testkit.NewTestKit(c, store) +func TestNDVGroupCols(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int not null, b int not null, key(a,b))") @@ -162,11 +138,12 @@ func (s *testStatsSuite) TestNDVGroupCols(c *C) { SQL string Plan []string } - s.testData.GetTestCases(c, &input, &output) + statsSuiteData := core.GetStatsSuiteData() + statsSuiteData.GetTestCases(t, &input, &output) for i, tt := range input { - s.testData.OnRecord(func() { + testdata.OnRecord(func() { output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) }) // The test point is the row count estimation for aggregations and joins. tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go index 3ca5cbfc66568..b356dbe4ca3da 100644 --- a/planner/core/stringer_test.go +++ b/planner/core/stringer_test.go @@ -38,7 +38,7 @@ func TestPlanStringer(t *testing.T) { }{ { sql: "show columns from t like 'a'", - plan: "Show(field:[a])", + plan: "Show(field_pattern:[(?i)a])", }, { sql: "show columns from t like 'a%'", @@ -56,6 +56,22 @@ func TestPlanStringer(t *testing.T) { sql: "desc t a", plan: "Show(field:[a])", }, + { + sql: "show tables in test like 't'", + plan: "Show(table:[t])", + }, + { + sql: "show tables in test like 'T'", + plan: "Show(table:[t])", + }, + { + sql: "show tables in test like 't%'", + plan: "Show(table_pattern:[(?i)t.*])", + }, + { + sql: "show tables in test like '%T%'", + plan: "Show(table_pattern:[(?i).*T.*])", + }, } parser := parser.New() for _, tt := range tests { diff --git a/planner/core/task.go b/planner/core/task.go index 0c0dfd00d9005..a321834be79ed 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -209,6 +209,20 @@ func (p *basePhysicalPlan) attach2Task(tasks ...task) task { func (p *PhysicalUnionScan) attach2Task(tasks ...task) task { p.cost = tasks[0].cost() + // We need to pull the projection under unionScan upon unionScan. + // Since the projection only prunes columns, it's ok the put it upon unionScan. + if sel, ok := tasks[0].plan().(*PhysicalSelection); ok { + if pj, ok := sel.children[0].(*PhysicalProjection); ok { + // Convert unionScan->selection->projection to projection->unionScan->selection. + sel.SetChildren(pj.children...) + p.SetChildren(sel) + p.stats = tasks[0].plan().statsInfo() + rt, _ := tasks[0].(*rootTask) + rt.p = p + pj.SetChildren(p) + return pj.attach2Task(tasks...) + } + } if pj, ok := tasks[0].plan().(*PhysicalProjection); ok { // Convert unionScan->projection to projection->unionScan, because unionScan can't handle projection as its children. p.SetChildren(pj.children...) @@ -1018,8 +1032,6 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { // the number of regions involved, we simply use DistSQLScanConcurrency. copIterWorkers := float64(t.plan().SCtx().GetSessionVars().DistSQLScanConcurrency()) t.finishIndexPlan() - needExtraProj := false - var prevSchema *expression.Schema // Network cost of transferring rows of table scan to TiDB. if t.tablePlan != nil { t.cst += t.count() * sessVars.GetNetworkFactor(nil) * t.tblColHists.GetAvgRowSize(ctx, t.tablePlan.Schema().Columns, false, false) @@ -1035,11 +1047,12 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { } ts := tp.(*PhysicalTableScan) prevColumnLen := len(ts.Columns) - prevSchema = ts.schema.Clone() + prevSchema := ts.schema.Clone() ts.Columns = ExpandVirtualColumn(ts.Columns, ts.schema, ts.Table.Columns) - if len(ts.Columns) > prevColumnLen { + if !t.needExtraProj && len(ts.Columns) > prevColumnLen { // Add an projection to make sure not to output extract columns. - needExtraProj = true + t.needExtraProj = true + t.originSchema = prevSchema } } t.cst /= copIterWorkers @@ -1106,9 +1119,9 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { aggPushedDown = true } - if needExtraProj && !aggPushedDown { - proj := PhysicalProjection{Exprs: expression.Column2Exprs(prevSchema.Columns)}.Init(ts.ctx, ts.stats, ts.SelectBlockOffset(), nil) - proj.SetSchema(prevSchema) + if t.needExtraProj && !aggPushedDown { + proj := PhysicalProjection{Exprs: expression.Column2Exprs(t.originSchema.Columns)}.Init(ts.ctx, ts.stats, ts.SelectBlockOffset(), nil) + proj.SetSchema(t.originSchema) proj.SetChildren(p) proj.cost = t.cost() newTask.p = proj @@ -1576,11 +1589,13 @@ type AggInfo struct { // BuildFinalModeAggregation splits either LogicalAggregation or PhysicalAggregation to finalAgg and partial1Agg, // returns the information of partial and final agg. -// partialIsCop means whether partial agg is a cop task. +// partialIsCop means whether partial agg is a cop task. When partialIsCop is false, +// we do not set the AggMode for partialAgg cause it may be split further when +// building the aggregate executor(e.g. buildHashAgg will split the AggDesc further for parallel executing). +// firstRowFuncMap is a map between partial first_row to final first_row, will be used in RemoveUnnecessaryFirstRow func BuildFinalModeAggregation( - sctx sessionctx.Context, original *AggInfo, partialIsCop bool, isMPPTask bool) (partial, final *AggInfo, funcMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { - - funcMap = make(map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc, len(original.AggFuncs)) + sctx sessionctx.Context, original *AggInfo, partialIsCop bool, isMPPTask bool) (partial, final *AggInfo, firstRowFuncMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) { + firstRowFuncMap = make(map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc, len(original.AggFuncs)) partial = &AggInfo{ AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(original.AggFuncs)), GroupByItems: original.GroupByItems, @@ -1772,20 +1787,23 @@ func BuildFinalModeAggregation( partial.Schema.Columns[partialCursor-1].RetType = sumAgg.RetTp partial.AggFuncs = append(partial.AggFuncs, cntAgg, sumAgg) } else if aggFunc.Name == ast.AggFuncApproxCountDistinct || aggFunc.Name == ast.AggFuncGroupConcat { - newAggFunc := *aggFunc + newAggFunc := aggFunc.Clone() newAggFunc.Name = aggFunc.Name newAggFunc.RetTp = partial.Schema.Columns[partialCursor-1].GetType() - partial.AggFuncs = append(partial.AggFuncs, &newAggFunc) + partial.AggFuncs = append(partial.AggFuncs, newAggFunc) if aggFunc.Name == ast.AggFuncGroupConcat { // append the last separator arg args = append(args, aggFunc.Args[len(aggFunc.Args)-1]) } } else { - partial.AggFuncs = append(partial.AggFuncs, aggFunc) + partialFuncDesc := aggFunc.Clone() + partial.AggFuncs = append(partial.AggFuncs, partialFuncDesc) + if aggFunc.Name == ast.AggFuncFirstRow { + firstRowFuncMap[partialFuncDesc] = finalAggFunc + } } finalAggFunc.Mode = aggregation.FinalMode - funcMap[aggFunc] = finalAggFunc } finalAggFunc.Args = args @@ -1793,6 +1811,11 @@ func BuildFinalModeAggregation( final.AggFuncs[i] = finalAggFunc } partial.Schema.Append(partialGbySchema.Columns...) + if partialIsCop { + for _, f := range partial.AggFuncs { + f.Mode = aggregation.Partial1Mode + } + } return } @@ -1871,7 +1894,7 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTas if !CheckAggCanPushCop(p.ctx, p.AggFuncs, p.GroupByItems, copTaskType) { return nil, p.self } - partialPref, finalPref, funcMap := BuildFinalModeAggregation(p.ctx, &AggInfo{ + partialPref, finalPref, firstRowFuncMap := BuildFinalModeAggregation(p.ctx, &AggInfo{ AggFuncs: p.AggFuncs, GroupByItems: p.GroupByItems, Schema: p.Schema().Clone(), @@ -1884,8 +1907,7 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTas } // Remove unnecessary FirstRow. partialPref.AggFuncs = RemoveUnnecessaryFirstRow(p.ctx, - finalPref.AggFuncs, finalPref.GroupByItems, - partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, funcMap) + finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, firstRowFuncMap) if copTaskType == kv.TiDB { // For partial agg of TiDB cop task, since TiDB coprocessor reuse the TiDB executor, // and TiDB aggregation executor won't output the group by value, @@ -1942,12 +1964,11 @@ func genFirstRowAggForGroupBy(ctx sessionctx.Context, groupByItems []expression. // Can optimize the schema to [count(b), a] , and change the index to get value. func RemoveUnnecessaryFirstRow( sctx sessionctx.Context, - finalAggFuncs []*aggregation.AggFuncDesc, finalGbyItems []expression.Expression, partialAggFuncs []*aggregation.AggFuncDesc, partialGbyItems []expression.Expression, partialSchema *expression.Schema, - funcMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) []*aggregation.AggFuncDesc { + firstRowFuncMap map[*aggregation.AggFuncDesc]*aggregation.AggFuncDesc) []*aggregation.AggFuncDesc { partialCursor := 0 newAggFuncs := make([]*aggregation.AggFuncDesc, 0, len(partialAggFuncs)) @@ -1967,7 +1988,7 @@ func RemoveUnnecessaryFirstRow( } if gbyExpr.Equal(sctx, aggFunc.Args[0]) { canOptimize = true - funcMap[aggFunc].Args[0] = finalGbyItems[j] + firstRowFuncMap[aggFunc].Args[0] = finalGbyItems[j] break } } diff --git a/planner/core/testdata/enforce_mpp_suite_in.json b/planner/core/testdata/enforce_mpp_suite_in.json index 70f519b80192f..41b3aac9920a6 100644 --- a/planner/core/testdata/enforce_mpp_suite_in.json +++ b/planner/core/testdata/enforce_mpp_suite_in.json @@ -28,21 +28,21 @@ "name": "TestEnforceMPPWarning1", "cases": [ "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;", - "explain select count(*) from t where a=1 -- 1. no replica", + "explain format = 'brief' select count(*) from t where a=1 -- 1. no replica", "cmd: create-replica", "explain select count(*) from t where a=1 -- 2. replica not ready", "cmd: enable-replica", "set @@session.tidb_isolation_read_engines = 'tikv';", "explain select count(*) from t where a=1 -- 3. isolation_engine not match", "set @@session.tidb_isolation_read_engines = 'tikv, tiflash';", - "explain select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", - "explain SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", - "EXPLAIN SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", - "EXPLAIN SELECT count(b) from t where a=1; -- 7. agg func has virtual column", - "EXPLAIN SELECT count(*) from t group by b; -- 8. group by virtual column", - "EXPLAIN SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", - "EXPLAIN SELECT count(a) from t where c=1; -- 11. type not supported", - "EXPLAIN SELECT count(a) from t where d=1; -- 11.1. type not supported" + "explain format = 'brief' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", + "explain format = 'brief' SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", + "EXPLAIN format = 'brief' SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", + "EXPLAIN format = 'brief' SELECT count(b) from t where a=1; -- 7. agg func has virtual column", + "EXPLAIN format = 'brief' SELECT count(*) from t group by b; -- 8. group by virtual column", + "EXPLAIN format = 'brief' SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", + "EXPLAIN format = 'brief' SELECT count(a) from t where c=1; -- 11. type not supported", + "EXPLAIN format = 'brief' SELECT count(a) from t where d=1; -- 11.1. type not supported" ] }, { @@ -82,7 +82,7 @@ "EXPLAIN SELECT * from t join s; -- can use mpp", "set @@tidb_broadcast_join_threshold_size = 104857600; set @@tidb_opt_broadcast_cartesian_join = 1;", "explain select a from t where t.a>1 or t.a in (select a from t); -- 7. left outer semi join", - "explain select a from t where t.a>1 or t.a not in (select a from t); -- 8. anti left outer semi join", + "explain select a from t where t.a>1 or t.a not in (select a from t); -- now it's supported -- 8. anti left outer semi join", "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions" ] } diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index d93fd15e63968..df32725b1090e 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -192,12 +192,12 @@ "Warn": null }, { - "SQL": "explain select count(*) from t where a=1 -- 1. no replica", + "SQL": "explain format = 'brief' select count(*) from t where a=1 -- 1. no replica", "Plan": [ - "StreamAgg_17 1.00 root funcs:count(Column#8)->Column#6", - "└─IndexReader_18 1.00 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#8", - " └─IndexRangeScan_16 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because there aren't tiflash replicas of table `t`." @@ -248,23 +248,23 @@ "Warn": null }, { - "SQL": "explain select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", + "SQL": "explain format = 'brief' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1 -- 4. hint use tikv", "Plan": [ - "StreamAgg_19 1.00 root funcs:count(Column#8)->Column#6", - "└─IndexReader_20 1.00 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 cop[tikv] funcs:count(1)->Column#8", - " └─IndexRangeScan_18 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#8", + " └─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." ] }, { - "SQL": "explain SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", + "SQL": "explain format = 'brief' SELECT a, ROW_NUMBER() OVER (ORDER BY a) FROM t; -- 5. window unsupported", "Plan": [ - "Window_7 10000.00 root row_number()->Column#7 over(order by test.t.a rows between current row and current row)", - "└─IndexReader_9 10000.00 root index:IndexFullScan_8", - " └─IndexFullScan_8 10000.00 cop[tikv] table:t, index:idx(a) keep order:true, stats:pseudo" + "Window 10000.00 root row_number()->Column#7 over(order by test.t.a rows between current row and current row)", + "└─IndexReader 10000.00 root index:IndexFullScan", + " └─IndexFullScan 10000.00 cop[tikv] table:t, index:idx(a) keep order:true, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because operator `Window` is not supported now.", @@ -272,27 +272,27 @@ ] }, { - "SQL": "EXPLAIN SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", + "SQL": "EXPLAIN format = 'brief' SELECT t1.b FROM t t1 join t t2 where t1.a=t2.a; -- 6. virtual column", "Plan": [ - "HashJoin_36 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─TableReader_56(Build) 9990.00 root data:Selection_55", - "│ └─Selection_55 9990.00 cop[tiflash] not(isnull(test.t.a))", - "│ └─TableFullScan_54 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - "└─TableReader_50(Probe) 9990.00 root data:Selection_49", - " └─Selection_49 9990.00 cop[tiflash] not(isnull(test.t.a))", - " └─TableFullScan_48 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because column `test.t.b` is a virtual column which is not supported now." ] }, { - "SQL": "EXPLAIN SELECT count(b) from t where a=1; -- 7. agg func has virtual column", + "SQL": "EXPLAIN format = 'brief' SELECT count(b) from t where a=1; -- 7. agg func has virtual column", "Plan": [ - "StreamAgg_11 1.00 root funcs:count(test.t.b)->Column#6", - "└─IndexLookUp_42 10.00 root ", - " ├─IndexRangeScan_40(Build) 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo", - " └─TableRowIDScan_41(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(test.t.b)->Column#6", + "└─IndexLookUp 10.00 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 10.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warn": [ "Aggregation can not be pushed to tiflash because expressions of AggFunc `count` contain virtual column or correlated column, which is not supported now", @@ -303,12 +303,12 @@ ] }, { - "SQL": "EXPLAIN SELECT count(*) from t group by b; -- 8. group by virtual column", + "SQL": "EXPLAIN format = 'brief' SELECT count(*) from t group by b; -- 8. group by virtual column", "Plan": [ - "HashAgg_6 8000.00 root group by:test.t.b, funcs:count(1)->Column#6", - "└─Projection_12 10000.00 root test.t.b", - " └─TableReader_11 10000.00 root data:TableFullScan_10", - " └─TableFullScan_10 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "HashAgg 8000.00 root group by:test.t.b, funcs:count(1)->Column#6", + "└─Projection 10000.00 root test.t.b", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ "Aggregation can not be pushed to tiflash because groupByItems contain virtual columns, which is not supported now", @@ -317,12 +317,12 @@ ] }, { - "SQL": "EXPLAIN SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t group by md5(a); -- 10. scalar func not supported", "Plan": [ - "HashAgg_6 8000.00 root group by:Column#8, funcs:count(Column#7)->Column#6", - "└─Projection_19 10000.00 root test.t.a, md5(cast(test.t.a, var_string(20)))->Column#8", - " └─TableReader_12 10000.00 root data:TableFullScan_10", - " └─TableFullScan_10 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "HashAgg 8000.00 root group by:Column#8, funcs:count(Column#7)->Column#6", + "└─Projection 10000.00 root test.t.a, md5(cast(test.t.a, var_string(20)))->Column#8", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ "Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", @@ -334,12 +334,12 @@ ] }, { - "SQL": "EXPLAIN SELECT count(a) from t where c=1; -- 11. type not supported", + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where c=1; -- 11. type not supported", "Plan": [ - "StreamAgg_11 1.00 root funcs:count(test.t.a)->Column#6", - "└─Selection_29 10.00 root eq(test.t.c, 1)", - " └─TableReader_28 10000.00 root data:TableFullScan_27", - " └─TableFullScan_27 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(test.t.a)->Column#6", + "└─Selection 10.00 root eq(test.t.c, 1)", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ "Expression about 'test.t.c' can not be pushed to TiFlash because it contains unsupported calculation of type 'enum'.", @@ -350,13 +350,13 @@ ] }, { - "SQL": "EXPLAIN SELECT count(a) from t where d=1; -- 11.1. type not supported", + "SQL": "EXPLAIN format = 'brief' SELECT count(a) from t where d=1; -- 11.1. type not supported", "Plan": [ - "HashAgg_9 1.00 root funcs:count(test.t.a)->Column#6", - "└─Selection_19 8000.00 root eq(test.t.d, 1)", - " └─TableReader_22 10000.00 root data:ExchangeSender_21", - " └─ExchangeSender_21 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan_20 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + "HashAgg 1.00 root funcs:count(test.t.a)->Column#6", + "└─Selection 8000.00 root eq(test.t.d, 1)", + " └─TableReader 10000.00 root data:ExchangeSender", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": [ "Expression about 'test.t.d' can not be pushed to TiFlash because it contains unsupported calculation of type 'bit'." @@ -588,34 +588,32 @@ { "SQL": "explain select a from t where t.a>1 or t.a in (select a from t); -- 7. left outer semi join", "Plan": [ - "Projection_7 8000.00 root test.t.a", - "└─Selection_9 8000.00 root or(gt(test.t.a, 1), Column#3)", - " └─MergeJoin_10 10000.00 root left outer semi join, left key:test.t.a, right key:test.t.a", - " ├─TableReader_30(Build) 10000.00 root data:TableFullScan_29", - " │ └─TableFullScan_29 10000.00 cop[tiflash] table:t keep order:true, stats:pseudo", - " └─TableReader_26(Probe) 10000.00 root data:TableFullScan_25", - " └─TableFullScan_25 10000.00 cop[tiflash] table:t keep order:true, stats:pseudo" + "TableReader_49 8000.00 root data:ExchangeSender_48", + "└─ExchangeSender_48 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 cop[tiflash] test.t.a", + " └─Selection_46 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_47 10000.00 cop[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_27(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender_26 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_25 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_24(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "MPP mode may be blocked because join type `left outer semi join` is not supported now.", - "MPP mode may be blocked because join type `left outer semi join` is not supported now." - ] + "Warn": null }, { - "SQL": "explain select a from t where t.a>1 or t.a not in (select a from t); -- 8. anti left outer semi join", + "SQL": "explain select a from t where t.a>1 or t.a not in (select a from t); -- now it's supported -- 8. anti left outer semi join", "Plan": [ - "Projection_7 8000.00 root test.t.a", - "└─Selection_9 8000.00 root or(gt(test.t.a, 1), Column#3)", - " └─MergeJoin_10 10000.00 root anti left outer semi join, left key:test.t.a, right key:test.t.a", - " ├─TableReader_30(Build) 10000.00 root data:TableFullScan_29", - " │ └─TableFullScan_29 10000.00 cop[tiflash] table:t keep order:true, stats:pseudo", - " └─TableReader_26(Probe) 10000.00 root data:TableFullScan_25", - " └─TableFullScan_25 10000.00 cop[tiflash] table:t keep order:true, stats:pseudo" + "TableReader_49 8000.00 root data:ExchangeSender_48", + "└─ExchangeSender_48 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 cop[tiflash] test.t.a", + " └─Selection_46 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#3)", + " └─HashJoin_47 10000.00 cop[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver_27(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender_26 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan_25 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan_24(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], - "Warn": [ - "MPP mode may be blocked because join type `anti left outer semi join` is not supported now.", - "MPP mode may be blocked because join type `anti left outer semi join` is not supported now." - ] + "Warn": null }, { "SQL": "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions", diff --git a/planner/core/testdata/integration_partition_suite_out.json b/planner/core/testdata/integration_partition_suite_out.json index de53ff095d7e9..726b0f6598d6d 100644 --- a/planner/core/testdata/integration_partition_suite_out.json +++ b/planner/core/testdata/integration_partition_suite_out.json @@ -378,16 +378,14 @@ { "SQL": "explain format = 'brief' select * from tlist where a in (0, 1, 2) and mod(a, 2)=0", "DynamicPlan": [ - "Selection 24.00 root eq(mod(list_partition_pruning.tlist.a, 2), 0)", - "└─TableReader 30.00 root partition:p0 data:Selection", - " └─Selection 30.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 1, 2)", - " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" + "TableReader 24.00 root partition:p0 data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tlist.a, 2), 0), in(list_partition_pruning.tlist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist keep order:false, stats:pseudo" ], "StaticPlan": [ - "Selection 24.00 root eq(mod(list_partition_pruning.tlist.a, 2), 0)", - "└─TableReader 30.00 root data:Selection", - " └─Selection 30.00 cop[tikv] in(list_partition_pruning.tlist.a, 0, 1, 2)", - " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" + "TableReader 24.00 root data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tlist.a, 2), 0), in(list_partition_pruning.tlist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tlist, partition:p0 keep order:false, stats:pseudo" ] }, { @@ -413,16 +411,14 @@ { "SQL": "explain format = 'brief' select * from tcollist where a in (0, 1, 2) and mod(a, 2)=0", "DynamicPlan": [ - "Selection 24.00 root eq(mod(list_partition_pruning.tcollist.a, 2), 0)", - "└─TableReader 30.00 root partition:p0 data:Selection", - " └─Selection 30.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 1, 2)", - " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" + "TableReader 24.00 root partition:p0 data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tcollist.a, 2), 0), in(list_partition_pruning.tcollist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist keep order:false, stats:pseudo" ], "StaticPlan": [ - "Selection 24.00 root eq(mod(list_partition_pruning.tcollist.a, 2), 0)", - "└─TableReader 30.00 root data:Selection", - " └─Selection 30.00 cop[tikv] in(list_partition_pruning.tcollist.a, 0, 1, 2)", - " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" + "TableReader 24.00 root data:Selection", + "└─Selection 24.00 cop[tikv] eq(mod(list_partition_pruning.tcollist.a, 2), 0), in(list_partition_pruning.tcollist.a, 0, 1, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:tcollist, partition:p0 keep order:false, stats:pseudo" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json deleted file mode 100644 index aece551a5b991..0000000000000 --- a/planner/core/testdata/integration_serial_suite_in.json +++ /dev/null @@ -1,424 +0,0 @@ -[ - { - "name": "TestSelPushDownTiFlash", - "cases": [ - "explain format = 'brief' select * from t where t.a > 1 and t.b = \"flash\" or t.a + 3 * t.a = 5", - "explain format = 'brief' select * from t where cast(t.a as double) + 3 = 5.1", - "explain format = 'brief' select * from t where b > 'a' order by convert(b, unsigned) limit 2", - "explain format = 'brief' select * from t where b > 'a' order by b limit 2" - ] - }, - { - "name": "TestVerboseExplain", - "cases": [ - "explain format = 'verbose' select count(*) from t3", - "explain format = 'verbose' select count(*) from t2", - "explain format = 'verbose' select * from t3 order by a", - "explain format = 'verbose' select * from t3 order by b", - "explain format = 'verbose' select * from t3 order by a limit 1", - "explain format = 'verbose' select * from t3 order by b limit 1", - "explain format = 'verbose' select count(*) from t2 group by a", - "explain format = 'verbose' select count(*) from t3 where b = 0", - "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", - "explain format = 'verbose' select count(*) from t2 where a = 0", - "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", - "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", - "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", - "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", - "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a" - ] - - }, - { - "name": "TestRegardNULLAsPoint", - "cases": [ - "select * from tuk where a<=>null and b=1", - "select * from tik where a<=>null and b=1", - "select * from tuk where a<=>null and b>0 and b<2", - "select * from tik where a<=>null and b>0 and b<2", - "select * from tuk where a<=>null and b>=1 and b<2", - "select * from tik where a<=>null and b>=1 and b<2", - "select * from tuk where a<=>null and b=1 and c=1", - "select * from tik where a<=>null and b=1 and c=1", - "select * from tuk where a=1 and b<=>null and c=1", - "select * from tik where a=1 and b<=>null and c=1", - "select * from tuk where a<=>null and b<=>null and c=1", - "select * from tik where a<=>null and b<=>null and c=1", - "select * from tuk where a<=>null and b<=>null and c<=>null", - "select * from tik where a<=>null and b<=>null and c<=>null" - ] - }, - { - "name": "TestPushDownToTiFlashWithKeepOrder", - "cases": [ - "explain format = 'brief' select max(a) from t", - "explain format = 'brief' select min(a) from t" - ] - }, - { - "name": "TestMPPJoin", - "cases": [ - "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" - ] - }, - { - "name": "TestMPPOuterJoinBuildSideForBroadcastJoin", - "cases": [ - "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "explain format = 'brief' select count(*) from b right join a on a.id = b.id" - ] - }, - { - "name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", - "cases": [ - "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "explain format = 'brief' select count(*) from b right join a on a.id = b.id" - ] - }, - { - "name": "TestMPPOuterJoinBuildSideForShuffleJoin", - "cases": [ - "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "explain format = 'brief' select count(*) from b right join a on a.id = b.id" - ] - }, - { - "name": "TestMPPShuffledJoin", - "cases": [ - "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d1_k = d2_t.value and fact_t.d1_k = d3_t.value", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", - "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" - ] - }, - { - "name": "TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError", - "cases": [ - "explain format = 'brief' select v from t3 as a left join (select t1.v1, t1.v2, t1.v1 + t1.v2 as v from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2) b on a.v1 = b.v1 and a.v2 = b.v2", - "explain format = 'brief' select count(*), t2.v1, t2.v2 from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2 group by t2.v1, t2.v2", - "explain format = 'brief' select count(*), t2.v1, t2.v2 from t3 left join t2 on t3.v1 = t2.v1 and t3.v2 = t2.v2 group by t2.v1, t2.v2" - ] - }, - { - "name": "TestJoinNotSupportedByTiFlash", - "cases": [ - "explain format = 'brief' select * from table_1 a, table_1 b where a.bit_col = b.bit_col", - "explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100", - "explain format = 'brief' select * from table_1 a right join table_1 b on a.id = b.id and dayofmonth(b.datetime_col) > 100", - "explain format = 'brief' select * from table_1 a join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > dayofmonth(b.datetime_col)" - ] - }, - { - "name": "TestMPPWithHashExchangeUnderNewCollation", - "cases": [ - "explain format = 'brief' select * from table_1 a, table_1 b where a.value = b.value", - "explain format = 'brief' select * from table_1 a, table_2 b where a.value = b.value", - "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and b.value = c.value", - "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and a.value = c.value", - "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_1 group by value", - "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_2 group by value" - ] - }, - { - "name": "TestMPPWithBroadcastExchangeUnderNewCollation", - "cases": [ - "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.id = b.id", - "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.value = b.value" - ] - }, - { - "name": "TestMPPAvgRewrite", - "cases": [ - "explain format = 'brief' select /*+ avg_to_cop() */ id, avg(value+1),avg(value) from table_1 group by id" - ] - }, - { - "name": "TestReadFromStorageHint", - "cases": [ - "desc format = 'brief' select avg(a) from t", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a) from t", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a+1) from t", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(isnull(a)) from t", - "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIKV[t2]) */ * from t t1, t t2 where t1.a = t2.a", - "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIFLASH[t2]) */ * from t t1, t t2 where t1.a = t2.a", - "desc format = 'brief' select * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", - "desc format = 'brief' select /*+ read_from_storage(tiflash[tt]) */ * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", - "desc format = 'brief' select * from ttt order by ttt.a desc", - "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a desc", - "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a", - "desc format = 'brief' select /*+ read_from_storage(tikv[t, ttt]) */ * from ttt", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt" - ] - }, - { - "name": "TestReadFromStorageHintAndIsolationRead", - "cases": [ - "desc format = 'brief' select /*+ read_from_storage(tikv[t], tiflash[t]) */ avg(a) from t", - "desc format = 'brief' select /*+ read_from_storage(tikv[t]) */ avg(a) from t", - "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t" - ] - }, - { - "name": "TestIsolationReadDoNotFilterSystemDB", - "cases": [ - "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", - "desc format = 'brief' select * from information_schema.tables", - "desc format = 'brief' select * from mysql.stats_meta" - ] - }, - { - "name": "TestIsolationReadTiFlashNotChoosePointGet", - "cases": [ - "explain format = 'brief' select * from t where t.a = 1", - "explain format = 'brief' select * from t where t.a in (1, 2)" - ] - }, - { - "name": "TestIsolationReadTiFlashUseIndexHint", - "cases": [ - "explain format = 'brief' select * from t", - "explain format = 'brief' select * from t use index();", - "explain format = 'brief' select /*+ use_index(t, idx)*/ * from t", - "explain format = 'brief' select /*+ use_index(t)*/ * from t" - ] - }, - { - "name": "TestIssue20710", - "cases": [ - "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", - "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b" - ] - }, - { - "name": "TestPushDownProjectionForTiFlash", - "cases": [ - "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select * from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "desc format = 'brief' select A.id from t as A where exists (select 1 from t where t.id=A.id)", - "desc format = 'brief' select A.id from t as A where not exists (select 1 from t where t.id=A.id)", - "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;" - ] - }, - { - "name": "TestPushDownProjectionForMPP", - "cases": [ - "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select B.b+A.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", - "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "desc format = 'brief' select id from t as A where exists (select 1 from t where t.id=A.id)", - "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)", - "desc format = 'brief' select b*2, id from (select avg(value+2) as b, id from t group by id) C order by id", - "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;" - ] - }, - { - "name": "TestMppUnionAll", - "cases": [ - "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", - "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", - "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", - "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", - "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt" - ] - }, - { - "name": "TestMppJoinDecimal", - "cases": [ - "desc format = 'brief' select t1.c1, t1.c2, t2.c1, t2.c2, t2.c3 from t t1 join t t2 on t1.c1 + 1 = t2.c2 - 10 and t1.c1 * 3 = t2.c3 / 2", - "desc format = 'brief' select * from (select c1, c2, c5, count(*) c from t group by c1, c2, c5) t1 join (select c1, c2, c3, count(*) c from t group by c1, c2, c3) t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c5 = t2.c1", - "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c2 and t1.c3 = t2.c3 and t1.c4 = t2.c4 and t1.c5 = t2.c5", - "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", - "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", - "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", - "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", - "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2" - ] - }, - { - "name": "TestPushDownAggForMPP", - "cases": [ - "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id+1 from t)A", - "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "desc format = 'brief' select count(*) from t", - "desc format = 'brief' select count(*), id from t group by id", - "desc format = 'brief' select count(*), id + 1 from t group by id + 1", - "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", - "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", - "desc format = 'brief' select avg(value) as b,id from t group by id", - "desc format = 'brief' select /*+hash_agg()*/ sum(b) from (select avg(value) as b, id from t group by id)A", - "desc format = 'brief' select id from t group by id having avg(value)>0", - "desc format = 'brief' select avg(value),id from t group by id having avg(value)>0", - "desc format = 'brief' select avg(value) +1,id from t group by id", - "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", - "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", - "desc format = 'brief' select count(distinct value),id from t group by id", - "desc format = 'brief' select count(distinct value),sum(distinct value),id from t group by id", - "desc format = 'brief' select * from t join ( select count(distinct value), id from t group by id) as A on A.id = t.id", - "desc format = 'brief' select * from t join ( select count(1/value), id from t group by id) as A on A.id = t.id", - "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", - "desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/", - "desc format = 'brief' select count(distinct value) from t", - "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", - "desc format = 'brief' select count(distinct value), count(value), avg(value) from t" - ] - }, - { - "name": "TestMppAggTopNWithJoin", - "cases": [ - "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", - "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", - "desc format = 'brief' select * from t join ( select count(*) as v, id from t group by value,id having value+v <10) as A on A.id = t.id", - "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", - "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", - "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", - "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", - "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", - "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", - "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", - "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", - "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", - "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1" - ] - }, - { - "name": "TestIndexMerge", - "cases": [ - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" - ] - }, - { - "name": "TestLimitIndexLookUpKeepOrder", - "cases": [ - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" - ] - }, - { - "name": "TestIssue23887", - "cases": [ - "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" - ] - }, - { - "name": "TestMergeContinuousSelections", - "cases": [ - "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;" - ] - }, - { - "name": "TestPushDownGroupConcatToTiFlash", - "cases": [ - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", - "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts" - ] - }, - { - "name": "TestRejectSortForMPP", - "cases": [ - "desc format = 'brief' select count(*) from (select * from t order by id)a group by name,id order by id", - "desc format = 'brief' select count(*) from (select * from t order by id)a group by name order by 1", - "desc format = 'brief' select count(*) from (select id,name from t group by id,name order by id,name)a group by name order by 1", - "desc format = 'brief' select * from (select id from t group by id order by id)a join t on a.id=t.id order by 1", - "desc format = 'brief' select * from (select * from t order by id)a join t on a.id=t.id order by 1", - "desc format = 'brief' select * from ((select id from t order by 1) union all (select id+1 from t order by 1))c", - "desc format = 'brief' select * from ((select count(*) from (select id,name from t order by id)a group by name,id order by id) union all (select id+1 from t order by 1))c", - "desc format = 'brief' select * from (select * from t order by id)a order by name" - ] - } -] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json deleted file mode 100644 index 112bb95e43c64..0000000000000 --- a/planner/core/testdata/integration_serial_suite_out.json +++ /dev/null @@ -1,4037 +0,0 @@ -[ - { - "Name": "TestSelPushDownTiFlash", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t where t.a > 1 and t.b = \"flash\" or t.a + 3 * t.a = 5", - "Plan": [ - "TableReader 8000.67 root data:Selection", - "└─Selection 8000.67 cop[tiflash] or(and(gt(test.t.a, 1), eq(test.t.b, \"flash\")), eq(plus(test.t.a, mul(3, test.t.a)), 5))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where cast(t.a as double) + 3 = 5.1", - "Plan": [ - "TableReader 8000.00 root data:Selection", - "└─Selection 8000.00 cop[tiflash] eq(plus(cast(test.t.a, double BINARY), 3), 5.1)", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b > 'a' order by convert(b, unsigned) limit 2", - "Plan": [ - "Projection 2.00 root test.t.a, test.t.b", - "└─TopN 2.00 root Column#4, offset:0, count:2", - " └─Projection 2.00 root test.t.a, test.t.b, cast(test.t.b, bigint(22) UNSIGNED BINARY)->Column#4", - " └─TableReader 2.00 root data:Projection", - " └─Projection 2.00 batchCop[tiflash] test.t.a, test.t.b", - " └─TopN 2.00 batchCop[tiflash] Column#3, offset:0, count:2", - " └─Projection 3333.33 batchCop[tiflash] test.t.a, test.t.b, cast(test.t.b, bigint(22) UNSIGNED BINARY)->Column#3", - " └─Selection 3333.33 batchCop[tiflash] gt(test.t.b, \"a\")", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where b > 'a' order by b limit 2", - "Plan": [ - "TopN 2.00 root test.t.b, offset:0, count:2", - "└─TableReader 2.00 root data:TopN", - " └─TopN 2.00 batchCop[tiflash] test.t.b, offset:0, count:2", - " └─Selection 3333.33 batchCop[tiflash] gt(test.t.b, \"a\")", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestVerboseExplain", - "Cases": [ - { - "SQL": "explain format = 'verbose' select count(*) from t3", - "Plan": [ - "StreamAgg_20 1.00 137.00 root funcs:count(Column#9)->Column#4", - "└─TableReader_21 1.00 9.68 root data:StreamAgg_8", - " └─StreamAgg_8 1.00 12.68 cop[tikv] funcs:count(1)->Column#9", - " └─TableFullScan_18 3.00 128.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t2", - "Plan": [ - "StreamAgg_25 1.00 69.50 root funcs:count(Column#7)->Column#4", - "└─TableReader_26 1.00 5.17 root data:StreamAgg_9", - " └─StreamAgg_9 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan_24 3.00 60.50 batchCop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t3 order by a", - "Plan": [ - "Sort_4 3.00 45.85 root test.t3.a", - "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 128.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t3 order by b", - "Plan": [ - "Sort_4 3.00 45.85 root test.t3.b", - "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 128.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", - "Plan": [ - "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", - "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", - " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", - "Plan": [ - "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", - "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", - " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t2 group by a", - "Plan": [ - "TableReader_24 3.00 3.33 root data:ExchangeSender_23", - "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", - " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_21 3.00 68.00 batchCop[tiflash] ", - " └─ExchangeSender_20 3.00 68.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: N/A]", - " └─TableFullScan_19 3.00 65.00 batchCop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", - "Plan": [ - "StreamAgg_10 1.00 1.33 root funcs:count(1)->Column#4", - "└─IndexReader_15 0.00 1.33 root index:IndexRangeScan_14", - " └─IndexRangeScan_14 0.00 20.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", - "Plan": [ - "StreamAgg_10 1.00 19.33 root funcs:count(test.t3.a)->Column#4", - "└─IndexLookUp_17 0.00 19.33 root ", - " ├─IndexRangeScan_15(Build) 0.00 20.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", - " └─TableRowIDScan_16(Probe) 0.00 20.00 cop[tikv] table:t3 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", - "Plan": [ - "StreamAgg_11 1.00 4.93 root funcs:count(1)->Column#4", - "└─TableReader_23 0.00 4.93 root data:Selection_22", - " └─Selection_22 0.00 74.00 cop[tiflash] eq(test.t2.a, 0)", - " └─TableFullScan_21 3.00 65.00 cop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", - "Plan": [ - "StreamAgg_10 1.00 60.22 root funcs:count(1)->Column#7", - "└─HashJoin_40 3.00 51.22 root inner join, equal:[eq(test.t3.a, test.t3.b)]", - " ├─IndexReader_28(Build) 3.00 11.66 root index:IndexFullScan_27", - " │ └─IndexFullScan_27 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_26(Probe) 3.00 10.76 root data:Selection_25", - " └─Selection_25 3.00 137.00 cop[tikv] not(isnull(test.t3.a))", - " └─TableFullScan_24 3.00 128.00 cop[tikv] table:t keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", - "Plan": [ - "StreamAgg_12 1.00 18.93 root funcs:count(1)->Column#7", - "└─TableReader_44 3.00 9.93 root data:ExchangeSender_43", - " └─ExchangeSender_43 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_40 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_19(Build) 3.00 77.00 cop[tiflash] ", - " │ └─ExchangeSender_18 3.00 77.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection_17 3.00 74.00 cop[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_16 3.00 65.00 cop[tiflash] table:t1 keep order:false", - " └─Selection_21(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_20 3.00 65.00 cop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", - "Plan": [ - "StreamAgg_15 1.00 60.60 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 51.60 root inner join, equal:[eq(test.t1.b, test.t3.b)]", - " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", - " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", - " │ └─ExchangeSender_34 3.00 106.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection_33 3.00 103.00 cop[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan_32 3.00 94.00 cop[tiflash] table:t1 keep order:false", - " └─Selection_37(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_36 3.00 65.00 cop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", - "Plan": [ - "HashJoin_19 3.00 127.40 root CARTESIAN left outer semi join", - "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", - "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", - "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", - "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 95.82 root 1->Column#28", - " └─Apply_22 3.00 76.02 root CARTESIAN left outer join", - " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", - " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", - " └─Projection_27(Probe) 1.00 21.95 root 1->Column#26", - " └─Limit_30 1.00 3.35 root offset:0, count:1", - " └─TableReader_38 1.00 3.35 root data:ExchangeSender_37", - " └─ExchangeSender_37 1.00 79.50 cop[tiflash] ExchangeType: PassThrough", - " └─Limit_36 1.00 79.50 cop[tiflash] offset:0, count:1", - " └─TableFullScan_35 1.00 79.50 cop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a", - "Plan": [ - "StreamAgg_11 1.00 59.65 root funcs:count(1)->Column#7", - "└─MergeJoin_29 3.00 50.65 root inner join, left key:test.t1.a, right key:test.t2.a", - " ├─Sort_27(Build) 3.00 20.83 root test.t2.a", - " │ └─TableReader_26 3.00 6.56 root data:Selection_25", - " │ └─Selection_25 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", - " │ └─TableFullScan_24 3.00 65.00 cop[tiflash] table:t2 keep order:false", - " └─Sort_20(Probe) 3.00 20.83 root test.t1.a", - " └─TableReader_19 3.00 6.56 root data:Selection_18", - " └─Selection_18 3.00 74.00 cop[tiflash] not(isnull(test.t1.a))", - " └─TableFullScan_17 3.00 65.00 cop[tiflash] table:t1 keep order:false" - ] - } - ] - }, - { - "Name": "TestRegardNULLAsPoint", - "Cases": [ - { - "SQL": "select * from tuk where a<=>null and b=1", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.01 root index:Selection_6", - "└─Selection_6 0.01 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b=1", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.01 root index:Selection_6", - "└─Selection_6 0.01 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b>0 and b<2", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b>0 and b<2", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b>=1 and b<2", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b>=1 and b<2", - "PlanEnabled": [ - "IndexReader_6 0.10 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.25 root index:Selection_6", - "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 ", - " 1 ", - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b=1 and c=1", - "PlanEnabled": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tuk.b, 1), eq(test.tuk.c, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b=1 and c=1", - "PlanEnabled": [ - "IndexReader_6 0.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tik.b, 1), eq(test.tik.c, 1)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1 1", - " 1 1" - ] - }, - { - "SQL": "select * from tuk where a=1 and b<=>null and c=1", - "PlanEnabled": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1)", - " └─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1 1", - "1 1" - ] - }, - { - "SQL": "select * from tik where a=1 and b<=>null and c=1", - "PlanEnabled": [ - "IndexReader_6 0.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1)", - " └─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" - ], - "Result": [ - "1 1", - "1 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b<=>null and c=1", - "PlanEnabled": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1), nulleq(test.tuk.b, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1", - " 1" - ] - }, - { - "SQL": "select * from tik where a<=>null and b<=>null and c=1", - "PlanEnabled": [ - "IndexReader_6 0.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1), nulleq(test.tik.b, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " 1", - " 1" - ] - }, - { - "SQL": "select * from tuk where a<=>null and b<=>null and c<=>null", - "PlanEnabled": [ - "IndexReader_6 1.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] nulleq(test.tuk.b, NULL), nulleq(test.tuk.c, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " ", - " " - ] - }, - { - "SQL": "select * from tik where a<=>null and b<=>null and c<=>null", - "PlanEnabled": [ - "IndexReader_6 0.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" - ], - "PlanDisabled": [ - "IndexReader_7 0.00 root index:Selection_6", - "└─Selection_6 0.00 cop[tikv] nulleq(test.tik.b, NULL), nulleq(test.tik.c, NULL)", - " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" - ], - "Result": [ - " ", - " " - ] - } - ] - }, - { - "Name": "TestPushDownToTiFlashWithKeepOrder", - "Cases": [ - { - "SQL": "explain format = 'brief' select max(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:max(test.t.a)->Column#3", - "└─TopN 1.00 root test.t.a:desc, offset:0, count:1", - " └─TableReader 1.00 root data:TopN", - " └─TopN 1.00 batchCop[tiflash] test.t.a:desc, offset:0, count:1", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select min(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:min(test.t.a)->Column#3", - "└─Limit 1.00 root offset:0, count:1", - " └─TableReader 1.00 root data:Limit", - " └─Limit 1.00 cop[tiflash] offset:0, count:1", - " └─TableFullScan 1.00 cop[tiflash] table:t keep order:true, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestMPPJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#17", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 8.00 root data:ExchangeSender", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", - " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#12", - "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#12", - "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#12", - "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#12", - "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 8.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─TableReader 6.40 root data:ExchangeSender", - " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 6.40 cop[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPOuterJoinBuildSideForBroadcastJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: N/A]", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: N/A]", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", - " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: N/A]", - " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: N/A]", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPOuterJoinBuildSideForShuffleJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: N/A]", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", - " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: N/A]", - " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#7", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: N/A]", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", - " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: N/A]", - " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", - " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPShuffledJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─ExchangeReceiver(Probe) 64.00 batchCop[tiflash] ", - " └─ExchangeSender 64.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: N/A]", - " └─HashJoin 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─ExchangeReceiver(Probe) 32.00 batchCop[tiflash] ", - " └─ExchangeSender 32.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: N/A]", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d1_k = d2_t.value and fact_t.d1_k = d3_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#22)->Column#19", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#22", - " └─HashJoin 204.80 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Projection(Probe) 102.40 batchCop[tiflash] test.fact_t.d1_k", - " └─Selection 102.40 batchCop[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", - " └─HashJoin 128.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", - " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: N/A]", - " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", - " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: N/A]", - " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError", - "Cases": [ - { - "SQL": "explain format = 'brief' select v from t3 as a left join (select t1.v1, t1.v2, t1.v1 + t1.v2 as v from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2) b on a.v1 = b.v1 and a.v2 = b.v2", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 cop[tiflash] Column#13", - " └─HashJoin 1.00 cop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t1.v1) eq(test.t3.v2, test.t1.v2)]", - " ├─ExchangeReceiver(Build) 1.00 cop[tiflash] ", - " │ └─ExchangeSender 1.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#23, collate: N/A], [name: Column#24, collate: N/A]", - " │ └─Projection 1.00 cop[tiflash] test.t3.v1, test.t3.v2, cast(test.t3.v1, decimal(20,2))->Column#23, cast(test.t3.v2, decimal(20,2))->Column#24", - " │ └─TableFullScan 1.00 cop[tiflash] table:a keep order:false", - " └─Projection(Probe) 2.00 cop[tiflash] test.t1.v1, test.t1.v2, plus(test.t1.v1, test.t1.v2)->Column#13", - " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: N/A], [name: test.t1.v2, collate: N/A]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.t1.v1)), not(isnull(test.t1.v2))", - " │ └─TableFullScan 2.00 cop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 cop[tiflash] ", - " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: N/A], [name: Column#16, collate: N/A]", - " └─Projection 8.00 cop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#15, cast(test.t2.v2, decimal(20,2))->Column#16", - " └─Selection 8.00 cop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 cop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2 group by t2.v1, t2.v2", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#22)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: N/A], [name: test.t2.v2, collate: N/A]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#22", - " └─HashJoin 2.00 batchCop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", - " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: N/A], [name: test.t1.v2, collate: N/A]", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", - " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: N/A], [name: Column#15, collate: N/A]", - " └─Projection 8.00 batchCop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#14, cast(test.t2.v2, decimal(20,2))->Column#15", - " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t3 left join t2 on t3.v1 = t2.v1 and t3.v2 = t2.v2 group by t2.v1, t2.v2", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#16)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: N/A], [name: test.t2.v2, collate: N/A]", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#16", - " └─HashJoin 1.00 batchCop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", - " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: N/A], [name: test.t3.v2, collate: N/A]", - " │ └─TableFullScan 1.00 batchCop[tiflash] table:t3 keep order:false", - " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", - " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: N/A], [name: test.t2.v2, collate: N/A]", - " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", - " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" - ] - } - ] - }, - { - "Name": "TestJoinNotSupportedByTiFlash", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from table_1 a, table_1 b where a.bit_col = b.bit_col", - "Plan": [ - "HashJoin 2.00 root inner join, equal:[eq(test.table_1.bit_col, test.table_1.bit_col)]", - "├─TableReader(Build) 2.00 root data:TableFullScan", - "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - "└─TableReader(Probe) 2.00 root data:TableFullScan", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100", - "Plan": [ - "HashJoin 2.00 root left outer join, equal:[eq(test.table_1.id, test.table_1.id)], left cond:[gt(dayofmonth(test.table_1.datetime_col), 100)]", - "├─TableReader(Build) 2.00 root data:TableFullScan", - "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - "└─TableReader(Probe) 2.00 root data:TableFullScan", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a right join table_1 b on a.id = b.id and dayofmonth(b.datetime_col) > 100", - "Plan": [ - "HashJoin 2.00 root right outer join, equal:[eq(test.table_1.id, test.table_1.id)], right cond:gt(dayofmonth(test.table_1.datetime_col), 100)", - "├─TableReader(Build) 2.00 root data:TableFullScan", - "│ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - "└─TableReader(Probe) 2.00 root data:TableFullScan", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > dayofmonth(b.datetime_col)", - "Plan": [ - "HashJoin 2.00 root inner join, equal:[eq(test.table_1.id, test.table_1.id)], other cond:gt(dayofmonth(test.table_1.datetime_col), dayofmonth(test.table_1.datetime_col))", - "├─TableReader(Build) 2.00 root data:TableFullScan", - "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - "└─TableReader(Probe) 2.00 root data:TableFullScan", - " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPWithHashExchangeUnderNewCollation", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from table_1 a, table_1 b where a.value = b.value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a, table_2 b where a.value = b.value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and b.value = c.value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_2.value, test.table_1.value)]", - " ├─HashJoin(Build) 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and a.value = c.value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " │ └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", - " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", - " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", - " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", - " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_1 group by value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_1.value", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_1.value)->test.table_1.value", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:count(1)->Column#7", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_2 group by value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_2.value", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_2.value)->test.table_2.value", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:count(1)->Column#7", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_2 keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPWithBroadcastExchangeUnderNewCollation", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.id = b.id", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.id, test.table_1.id)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─TableFullScan(Probe) 2.00 cop[tiflash] table:b keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.value = b.value", - "Plan": [ - "TableReader 2.00 root data:ExchangeSender", - "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", - " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", - " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", - " └─Selection(Probe) 2.00 cop[tiflash] not(isnull(test.table_1.value))", - " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" - ] - } - ] - }, - { - "Name": "TestMPPAvgRewrite", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ avg_to_cop() */ id, avg(value+1),avg(value) from table_1 group by id", - "Plan": [ - "Projection 2.00 root test.table_1.id, Column#4, Column#5", - "└─TableReader 2.00 root data:ExchangeSender", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 2.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#4, div(Column#5, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#5, test.table_1.id", - " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.id, funcs:sum(Column#15)->Column#13, funcs:sum(Column#16)->Column#4, funcs:sum(Column#17)->Column#14, funcs:sum(Column#18)->Column#5, funcs:firstrow(test.table_1.id)->test.table_1.id", - " └─ExchangeReceiver 2.00 batchCop[tiflash] ", - " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.id, collate: binary]", - " └─HashAgg 2.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#16, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#18", - " └─Projection 2.00 batchCop[tiflash] plus(test.table_1.value, 1)->Column#25, plus(test.table_1.value, 1)->Column#26, test.table_1.value, test.table_1.value, test.table_1.id", - " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" - ] - } - ] - }, - { - "Name": "TestReadFromStorageHint", - "Cases": [ - { - "SQL": "desc format = 'brief' select avg(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a+1) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.a, 1), decimal(20,0) BINARY)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(isnull(a)) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", - " └─Projection 10000.00 batchCop[tiflash] cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIKV[t2]) */ * from t t1, t t2 where t1.a = t2.a", - "Plan": [ - "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIFLASH[t2]) */ * from t t1, t t2 where t1.a = t2.a", - "Plan": [ - "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", - "Plan": [ - "TableReader 44.00 root data:TableRangeScan", - "└─TableRangeScan 44.00 cop[tiflash] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[tt]) */ * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", - "Plan": [ - "TableReader 44.00 root data:TableRangeScan", - "└─TableRangeScan 44.00 cop[tiflash] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select * from ttt order by ttt.a desc", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:ttt keep order:true, desc, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a desc", - "Plan": [ - "Sort 10000.00 root test.ttt.a:desc", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:true, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t, ttt]) */ * from ttt", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:ttt keep order:false, stats:pseudo" - ], - "Warn": [ - "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ READ_FROM_STORAGE(tikv[t, ttt]) */. Maybe you can use the table alias name" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:false, stats:pseudo" - ], - "Warn": [ - "[planner:1815]There are no matching table names for (t, tt) in optimizer hint /*+ READ_FROM_STORAGE(tiflash[t, ttt], tikv[tt]) */. Maybe you can use the table alias name" - ] - } - ] - }, - { - "Name": "TestReadFromStorageHintAndIsolationRead", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t], tiflash[t]) */ avg(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", - " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" - ], - "Warn": [ - "[planner:1815]Storage hints are conflict, you can only specify one storage type of table test.t" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t]) */ avg(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", - " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", - "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─IndexReader 1.00 root index:StreamAgg", - " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", - " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" - ], - "Warn": [ - "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{}])" - ] - } - ] - }, - { - "Name": "TestIsolationReadDoNotFilterSystemDB", - "Cases": [ - { - "SQL": "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", - "Plan": [ - "MemTableScan 10000.00 root table:tidb_query_duration PromQL:histogram_quantile(0.9, sum(rate(tidb_server_handle_query_duration_seconds_bucket{}[60s])) by (le,sql_type,instance)), start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s" - ] - }, - { - "SQL": "desc format = 'brief' select * from information_schema.tables", - "Plan": [ - "MemTableScan 10000.00 root table:TABLES " - ] - }, - { - "SQL": "desc format = 'brief' select * from mysql.stats_meta", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIsolationReadTiFlashNotChoosePointGet", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t where t.a = 1", - "Result": [ - "TableReader 1.00 root data:TableRangeScan", - "└─TableRangeScan 1.00 cop[tiflash] table:t range:[1,1], keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select * from t where t.a in (1, 2)", - "Result": [ - "TableReader 2.00 root data:TableRangeScan", - "└─TableRangeScan 2.00 cop[tiflash] table:t range:[1,1], [2,2], keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIsolationReadTiFlashUseIndexHint", - "Cases": [ - { - "SQL": "explain format = 'brief' select * from t", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "explain format = 'brief' select * from t use index();", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - }, - { - "SQL": "explain format = 'brief' select /*+ use_index(t, idx)*/ * from t", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": [ - "TiDB doesn't support index in the isolation read engines(value: 'tiflash')" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ use_index(t)*/ * from t", - "Plan": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ], - "Warn": null - } - ] - }, - { - "Name": "TestIssue20710", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", - "Plan": [ - "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", - "Plan": [ - "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", - "Plan": [ - "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", - "├─TableReader(Build) 9980.01 root data:Selection", - "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b", - "Plan": [ - "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─IndexLookUp(Probe) 1.25 root ", - " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", - " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", - " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", - " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestPushDownProjectionForTiFlash", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#6", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#6", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#8)->Column#6", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", - "Plan": [ - "StreamAgg 1.00 root funcs:count(Column#7)->Column#6", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", - "Plan": [ - "StreamAgg 1.00 root funcs:sum(Column#8)->Column#6", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "Plan": [ - "TableReader 10000.00 root data:HashJoin", - "└─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 10000.00 root data:HashJoin", - "└─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", - " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 10000.00 root data:HashJoin", - "└─HashJoin 10000.00 cop[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", - " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 12487.50 root data:HashJoin", - "└─HashJoin 12487.50 cop[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", - " ├─Selection(Build) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─Projection(Probe) 10000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "Plan": [ - "Projection 10000.00 root Column#10, Column#5", - "└─TableReader 10000.00 root data:HashJoin", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select A.id from t as A where exists (select 1 from t where t.id=A.id)", - "Plan": [ - "TableReader 7992.00 root data:HashJoin", - "└─HashJoin 7992.00 cop[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─Selection(Build) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select A.id from t as A where not exists (select 1 from t where t.id=A.id)", - "Plan": [ - "TableReader 8000.00 root data:HashJoin", - "└─HashJoin 8000.00 cop[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─TableFullScan(Build) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;", - "Plan": [ - "Projection 10000.00 root from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestPushDownProjectionForMPP", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#9)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#9)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#10)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#10", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#9)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#10)->Column#6", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#10", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select B.b+A.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "Plan": [ - "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] plus(Column#5, Column#10)->Column#11", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 10000.00 cop[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", - "Plan": [ - "TableReader 12487.50 root data:ExchangeSender", - "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 10000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", - "Plan": [ - "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] Column#10, Column#5", - " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", - " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", - " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", - " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select id from t as A where exists (select 1 from t where t.id=A.id)", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 cop[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8000.00 cop[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select b*2, id from (select avg(value+2) as b, id from t group by id) C order by id", - "Plan": [ - "Sort 8000.00 root test.t.id", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] mul(Column#5, 2)->Column#6, test.t.id", - " └─Projection 8000.00 batchCop[tiflash] div(Column#5, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#5, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#21)->Column#20, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#26, funcs:count(Column#24)->Column#21, funcs:sum(Column#25)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.value, 2)->Column#24, plus(test.t.value, 2)->Column#25, test.t.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;", - "Plan": [ - "TableReader 10000.00 root data:ExchangeSender", - "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 10000.00 cop[tiflash] from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestMppUnionAll", - "Cases": [ - { - "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#12", - " └─Union 20000.00 cop[tiflash] ", - " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#9, test.t.b", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#10", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#16)->Column#15", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#16", - " └─Union 20000.00 cop[tiflash] ", - " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#13, test.t.b", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#14", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#14)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#14", - " └─Union 20000.00 cop[tiflash] ", - " ├─Projection 10000.00 cop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", - " │ └─Projection 10000.00 cop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.c, double BINARY)->Column#10", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#14)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#14", - " └─Union 10000.00 batchCop[tiflash] ", - " └─Projection 10000.00 batchCop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt", - "Plan": [ - "StreamAgg 1.00 root funcs:count(1)->Column#11", - "└─Union 0.00 root ", - " ├─Projection 0.00 root test.t.a, cast(test.t.b, double BINARY)->Column#10", - " │ └─TableDual 0.00 root rows:0", - " └─Projection 0.00 root test.t1.a, cast(test.t1.c, double BINARY)->Column#10", - " └─TableDual 0.00 root rows:0" - ] - } - ] - }, - { - "Name": "TestMppJoinDecimal", - "Cases": [ - { - "SQL": "desc format = 'brief' select t1.c1, t1.c2, t2.c1, t2.c2, t2.c3 from t t1 join t t2 on t1.c1 + 1 = t2.c2 - 10 and t1.c1 * 3 = t2.c3 / 2", - "Plan": [ - "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c1, test.t.c2, test.t.c3", - "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#21, collate: N/A], [name: Column#15, collate: N/A]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(34,8) BINARY)->Column#21", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, mul(test.t.c1, 3)->Column#13, plus(test.t.c1, 1)->Column#15", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#22, collate: N/A], [name: Column#16, collate: N/A]", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, Column#14, Column#16, cast(Column#14, decimal(34,8) BINARY)->Column#22", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, div(test.t.c3, 2)->Column#14, minus(test.t.c2, 10)->Column#16", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select c1, c2, c5, count(*) c from t group by c1, c2, c5) t1 join (select c1, c2, c3, count(*) c from t group by c1, c2, c3) t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c5 = t2.c1", - "Plan": [ - "Projection 7976.02 root test.t.c1, test.t.c2, test.t.c5, Column#7, test.t.c1, test.t.c2, test.t.c3, Column#14", - "└─TableReader 7976.02 root data:ExchangeSender", - " └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", - " ├─ExchangeReceiver(Build) 7976.02 batchCop[tiflash] ", - " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A], [name: Column#31, collate: N/A], [name: test.t.c5, collate: N/A]", - " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", - " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5", - " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:sum(Column#15)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", - " │ └─ExchangeReceiver 7976.02 batchCop[tiflash] ", - " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A], [name: test.t.c2, collate: N/A], [name: test.t.c5, collate: N/A]", - " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#15", - " │ └─Selection 9970.03 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 7984.01 batchCop[tiflash] ", - " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A], [name: Column#32, collate: N/A], [name: Column#33, collate: N/A]", - " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3, cast(test.t.c3, decimal(10,5))->Column#32, cast(test.t.c1, decimal(40,20))->Column#33", - " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3", - " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:sum(Column#23)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", - " └─ExchangeReceiver 7984.01 batchCop[tiflash] ", - " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A], [name: test.t.c3, collate: N/A], [name: test.t.c1, collate: N/A]", - " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#23", - " └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c2 and t1.c3 = t2.c3 and t1.c4 = t2.c4 and t1.c5 = t2.c5", - "Plan": [ - "TableReader 12462.54 root data:ExchangeSender", - "└─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c2) eq(test.t.c3, test.t.c3) eq(test.t.c4, test.t.c4) eq(test.t.c5, test.t.c5)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A], [name: test.t.c2, collate: N/A], [name: test.t.c3, collate: N/A], [name: test.t.c4, collate: N/A], [name: test.t.c5, collate: N/A]", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A], [name: test.t.c2, collate: N/A], [name: test.t.c3, collate: N/A], [name: test.t.c4, collate: N/A], [name: test.t.c5, collate: N/A]", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", - "Plan": [ - "Projection 12462.54 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - "└─TableReader 12462.54 root data:ExchangeSender", - " └─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A], [name: Column#13, collate: N/A], [name: Column#15, collate: N/A], [name: test.t.c3, collate: N/A], [name: test.t.c5, collate: N/A]", - " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A], [name: Column#14, collate: N/A], [name: Column#16, collate: N/A], [name: test.t.c4, collate: N/A], [name: Column#17, collate: N/A]", - " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#14, cast(test.t.c3, decimal(10,5))->Column#16, cast(test.t.c1, decimal(40,20))->Column#17", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", - "Plan": [ - "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: N/A]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, plus(test.t.c1, test.t.c2)->Column#13", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: N/A]", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, div(test.t.c2, test.t.c3)->Column#14", - " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", - "Plan": [ - "Projection 7984.01 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - "└─TableReader 7984.01 root data:ExchangeSender", - " └─ExchangeSender 7984.01 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", - " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", - " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A], [name: Column#14, collate: N/A], [name: Column#16, collate: N/A], [name: test.t.c3, collate: N/A], [name: test.t.c5, collate: N/A]", - " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", - " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", - " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A], [name: Column#13, collate: N/A], [name: Column#15, collate: N/A], [name: test.t.c4, collate: N/A], [name: Column#17, collate: N/A]", - " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#13, cast(test.t.c3, decimal(10,5))->Column#15, cast(test.t.c1, decimal(40,20))->Column#17", - " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", - "Plan": [ - "Projection 19492.21 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - "└─TableReader 19492.21 root data:ExchangeSender", - " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#27, collate: N/A]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo", - " └─Projection(Probe) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", - " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#25, collate: N/A]", - " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c5, collate: N/A]", - " └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: N/A]", - " │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: N/A]", - " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: N/A], [name: test.tt.col_char_64_not_null, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", - " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", - " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: N/A]", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: N/A]", - " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestPushDownAggForMPP", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#8)->Column#5", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id+1 from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#7)->Column#5", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", - "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#8)->Column#5", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*) from t", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#6)->Column#4", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*), id from t group by id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*), id + 1 from t group by id + 1", - "Plan": [ - "Projection 8000.00 root Column#4, plus(test.t.id, 1)->Column#5", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:sum(Column#11)->Column#4, funcs:firstrow(Column#12)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:count(1)->Column#11, funcs:firstrow(Column#16)->Column#12", - " └─Projection 10000.00 batchCop[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", - "Plan": [ - "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", - "Plan": [ - "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", - "├─HashAgg(Build) 1.00 root funcs:count(Column#11)->Column#7", - "│ └─TableReader 1.00 root data:ExchangeSender", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#11", - "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select avg(value) as b,id from t group by id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#9, 0), 1, Column#9), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#10)->Column#9, funcs:sum(Column#11)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#10, funcs:sum(test.t.value)->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(b) from (select avg(value) as b, id from t group by id)A", - "Plan": [ - "HashAgg 1.00 root funcs:sum(Column#20)->Column#5", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#4)->Column#20", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#4", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#18)->Column#17, funcs:sum(Column#19)->Column#4", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#18, funcs:sum(test.t.value)->Column#19", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select id from t group by id having avg(value)>0", - "Plan": [ - "Projection 6400.00 root test.t.id", - "└─Selection 6400.00 root gt(Column#4, 0)", - " └─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#10, funcs:sum(Column#12)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#11, funcs:sum(test.t.value)->Column#12", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select avg(value),id from t group by id having avg(value)>0", - "Plan": [ - "Selection 6400.00 root gt(Column#4, 0)", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select avg(value) +1,id from t group by id", - "Plan": [ - "Projection 8000.00 root plus(Column#4, 1)->Column#5, test.t.id", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 7992.00 batchCop[tiflash] Column#7", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#17", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(distinct value),id from t group by id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(distinct value),sum(distinct value),id from t group by id", - "Plan": [ - "HashAgg 8000.00 root group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:sum(distinct test.t.value)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - "└─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select count(distinct value), id from t group by id) as A on A.id = t.id", - "Plan": [ - "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select count(1/value), id from t group by id) as A on A.id = t.id", - "Plan": [ - "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", - " │ └─Projection 9990.00 batchCop[tiflash] div(1, test.t.value)->Column#18, test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", - "Plan": [ - "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 6400.00 batchCop[tiflash] Column#4", - " └─HashAgg 6400.00 batchCop[tiflash] group by:Column#22, funcs:sum(Column#21)->Column#4", - " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#21, test.t.value", - " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", - " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: N/A]", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", - " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/", - "Plan": [ - "TableReader 6400.00 root data:ExchangeSender", - "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 6400.00 batchCop[tiflash] Column#8", - " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#8", - " └─Selection 8000.00 batchCop[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))", - " └─HashJoin 10000.00 batchCop[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", - " ├─Projection(Build) 7984.01 batchCop[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 7984.01 batchCop[tiflash] ", - " │ └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24", - " │ └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:B keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(distinct value) from t", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#4", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#4)->Column#5", - " └─Projection 1.00 batchCop[tiflash] Column#4", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(distinct value), count(value), avg(value) from t", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#4, Column#5, div(Column#6, cast(case(eq(Column#7, 0), 1, Column#7), decimal(20,0) BINARY))->Column#6", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#7, funcs:sum(Column#10)->Column#6", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, funcs:count(test.t.value)->Column#8, funcs:count(test.t.value)->Column#9, funcs:sum(test.t.value)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestMppAggTopNWithJoin", - "Cases": [ - { - "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", - "Plan": [ - "TableReader 9990.00 root data:ExchangeSender", - "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, Column#8)]", - " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 6400.00 batchCop[tiflash] plus(Column#7, test.t.id)->Column#8", - " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(Column#7, test.t.id)))", - " │ └─Projection 8000.00 batchCop[tiflash] Column#7, test.t.id", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#11", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select count(*) as v, id from t group by value,id having value+v <10) as A on A.id = t.id", - "Plan": [ - "Projection 7992.00 root test.t.id, test.t.value, Column#7, test.t.id", - "└─TableReader 7992.00 root data:ExchangeSender", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 6393.60 batchCop[tiflash] ", - " │ └─ExchangeSender 6393.60 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 6393.60 batchCop[tiflash] lt(plus(test.t.value, cast(Column#7, decimal(20,0) BINARY)), 10)", - " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#10)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: N/A], [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#10", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", - "Plan": [ - "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", - "├─HashAgg(Build) 1.00 root funcs:count(Column#10)->Column#7", - "│ └─TableReader 1.00 root data:ExchangeSender", - "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#10", - "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - "└─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 7992.00 batchCop[tiflash] Column#7", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#8", - " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#7, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", - "Plan": [ - "TableReader 7992.00 root data:ExchangeSender", - "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", - " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#17)->Column#11, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", - " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#16)->Column#13", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#16", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", - "Plan": [ - "TopN 1.00 root test.t.value, offset:0, count:1", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TopN 1.00 batchCop[tiflash] test.t.value, offset:0, count:1", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", - "Plan": [ - "Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value", - "└─TopN 1.00 root Column#8, offset:0, count:1", - " └─Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#8", - " └─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value", - " └─TopN 1.00 batchCop[tiflash] Column#7, offset:0, count:1", - " └─Projection 12487.50 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", - "Plan": [ - "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", - "└─TopN 20.00 root test.t.value, offset:0, count:20", - " └─TableReader 20.00 root data:ExchangeSender", - " └─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TopN 20.00 batchCop[tiflash] test.t.value, offset:0, count:20", - " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", - "Plan": [ - "Limit 1.00 root offset:0, count:1", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 1.00 cop[tiflash] offset:0, count:1", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", - "Plan": [ - "Limit 1.00 root offset:0, count:1", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 1.00 cop[tiflash] offset:0, count:1", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1", - "Plan": [ - "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", - "└─Limit 20.00 root offset:0, count:20", - " └─TableReader 20.00 root data:ExchangeSender", - " └─ExchangeSender 20.00 cop[tiflash] ExchangeType: PassThrough", - " └─Limit 20.00 cop[tiflash] offset:0, count:20", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 16.02 cop[tiflash] table:t1 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIndexMerge", - "Cases": [ - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", - "Plan": [ - "IndexMerge 8.00 root ", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 0.80 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root ", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 0.80 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", - "Plan": [ - "IndexMerge 8.00 root ", - "├─Selection(Build) 0.80 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─Selection(Build) 0.80 cop[tikv] 1", - "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" - ], - "Warnings": null - }, - { - "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", - "Plan": [ - "IndexMerge 0.29 root ", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", - "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", - "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", - " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" - ], - "Warnings": null - } - ] - }, - { - "Name": "TestLimitIndexLookUpKeepOrder", - "Cases": [ - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", - "Plan": [ - "Limit 0.00 root offset:0, count:10", - "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", - " └─IndexLookUp 0.00 root ", - " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", - " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", - " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestIssue23887", - "Cases": [ - { - "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", - "Plan": [ - "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.t.b)", - "├─TableReader(Build) 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─Projection(Probe) 10000.00 root 1->Column#27", - " └─Apply 10000.00 root CARTESIAN left outer join", - " ├─TableReader(Build) 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - " └─Projection(Probe) 1.00 root 1->Column#25", - " └─Limit 1.00 root offset:0, count:1", - " └─TableReader 1.00 root data:Limit", - " └─Limit 1.00 cop[tikv] offset:0, count:1", - " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo" - ], - "Res": [ - "1", - "1" - ] - } - ] - }, - { - "Name": "TestMergeContinuousSelections", - "Cases": [ - { - "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", - "Plan": [ - "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", - "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", - " ├─Selection(Build) 0.80 root ne(Column#27, 0)", - " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", - " │ └─TableReader 1.00 root data:ExchangeSender", - " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", - " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 12487.50 root data:ExchangeSender", - " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))", - " └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" - ] - } - ] - }, - { - "Name": "TestPushDownGroupConcatToTiFlash", - "Cases": [ - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", - "Plan": [ - "HashAgg 1.00 root funcs:group_concat(Column#7 separator \",\")->Column#5", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#7", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#11", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.col_1", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 order by Column#17 separator \",\")->Column#5, funcs:sum(Column#18)->Column#6, funcs:max(Column#19)->Column#7", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#16, test.ts.col_0, Column#12, Column#13", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#12, funcs:max(test.ts.col_0)->Column#13", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#19, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#24, 0), 1, Column#24), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#44, funcs:group_concat(distinct Column#36, Column#37, Column#38 order by Column#39 separator \",\")->Column#5, funcs:sum(Column#40)->Column#6, funcs:max(Column#41)->Column#7, funcs:sum(Column#42)->Column#24, funcs:sum(Column#43)->Column#8", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#38, test.ts.col_0, Column#25, Column#26, Column#27, Column#28, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#32, Column#33, Column#34, Column#35, funcs:count(1)->Column#25, funcs:max(Column#29)->Column#26, funcs:count(Column#30)->Column#27, funcs:sum(Column#31)->Column#28", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#31, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#18", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#30, Column#31, Column#32 order by Column#33 separator \",\")->Column#5, funcs:sum(Column#34)->Column#6, funcs:max(Column#35)->Column#7, funcs:sum(Column#36)->Column#18, funcs:sum(Column#37)->Column#8", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#32, test.ts.col_0, Column#19, Column#20, Column#21, Column#22", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#27, 0), 1, Column#27), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:min(Column#30)->Column#7, funcs:sum(Column#31)->Column#27, funcs:sum(Column#32)->Column#8", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(Column#33, Column#34, Column#35 separator \",\")->Column#28, funcs:count(Column#36)->Column#29, funcs:min(Column#37)->Column#30, funcs:count(Column#38)->Column#31, funcs:sum(Column#39)->Column#32", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#39, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#24, 0), 1, Column#24), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#44, funcs:group_concat(distinct Column#37, Column#38, Column#39 separator \",\")->Column#5, funcs:sum(Column#40)->Column#6, funcs:max(Column#41)->Column#7, funcs:sum(Column#42)->Column#24, funcs:sum(Column#43)->Column#8", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#39, Column#25, Column#26, Column#27, Column#28, test.ts.col_2", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#33, Column#34, Column#35, Column#36, funcs:count(Column#29)->Column#25, funcs:max(Column#30)->Column#26, funcs:count(Column#31)->Column#27, funcs:sum(Column#32)->Column#28", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#32, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", - "Plan": [ - "HashAgg 1.00 root funcs:group_concat(Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:avg(Column#17, Column#18)->Column#8", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#24, Column#25, Column#26 separator \",\")->Column#14, funcs:count(Column#27)->Column#15, funcs:min(Column#28)->Column#16, funcs:count(Column#29)->Column#17, funcs:sum(Column#30)->Column#18", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#30, Column#31, Column#32 separator \",\")->Column#5, funcs:sum(Column#33)->Column#6, funcs:max(Column#34)->Column#7, funcs:sum(Column#35)->Column#18, funcs:sum(Column#36)->Column#8", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#32, Column#19, Column#20, Column#21, Column#22", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", - " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:group_concat(Column#21, Column#22, Column#23 separator \",\")->Column#5, funcs:count(Column#24)->Column#6, funcs:group_concat(Column#25 order by Column#26 separator \",\")->Column#7, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#25, funcs:group_concat(distinct Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:count(Column#23)->Column#14, funcs:sum(Column#24)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24, test.ts.col_2", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: N/A]", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#18, Column#19 separator \",\")->Column#5, funcs:count(Column#20)->Column#6, funcs:group_concat(Column#21, Column#22 order by Column#23, Column#24 separator \",\")->Column#7, funcs:min(Column#25)->Column#8, funcs:count(Column#26)->Column#15, funcs:sum(Column#27)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#15, Column#16, Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:max(Column#23)->Column#8, funcs:count(Column#24)->Column#13, funcs:sum(Column#25)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#17, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#25", - " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", - " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#33, Column#34, Column#35 separator \",\")->Column#5, funcs:count(distinct Column#36)->Column#6, funcs:group_concat(Column#37 separator \",\")->Column#7, funcs:max(Column#38)->Column#8, funcs:sum(Column#39)->Column#19, funcs:sum(Column#40)->Column#9", - " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.col_2, Column#20, Column#21, Column#22, Column#23", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#29, Column#30, Column#31, Column#32, funcs:group_concat(Column#24, Column#25 separator \",\")->Column#20, funcs:max(Column#26)->Column#21, funcs:count(Column#27)->Column#22, funcs:sum(Column#28)->Column#23", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#25, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#25, 0), 1, Column#25), decimal(20,0) BINARY))->Column#9", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#47, funcs:group_concat(distinct Column#39, Column#40, Column#41 separator \",\")->Column#5, funcs:count(distinct Column#42)->Column#6, funcs:group_concat(Column#43 separator \",\")->Column#7, funcs:max(Column#44)->Column#8, funcs:sum(Column#45)->Column#25, funcs:sum(Column#46)->Column#9", - " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#41, test.ts.col_2, Column#26, Column#27, Column#28, Column#29, test.ts.col_0", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_0, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#35, Column#36, Column#37, Column#38, funcs:group_concat(Column#30, Column#31 separator \",\")->Column#26, funcs:max(Column#32)->Column#27, funcs:count(Column#33)->Column#28, funcs:sum(Column#34)->Column#29", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#34, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, Column#11", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, Column#11", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"01\", 0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#9, Column#10 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#8, var_string(20))->Column#9, cast(Column#8, var_string(20))->Column#10", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(distinct Column#18, Column#19 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#16, var_string(20))->Column#18, cast(Column#17, var_string(20))->Column#19, Column#15", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: N/A]", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, 10, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#16, funcs:group_concat(distinct Column#14, Column#15 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#13, var_string(20))->Column#14, cast(Column#13, var_string(20))->Column#15, Column#12", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", - " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(distinct Column#18, Column#19 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] cast(Column#16, var_string(20))->Column#18, Column#17, Column#15", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: N/A]", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, 1, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'", - "[types:1292]Truncated incorrect DOUBLE value: 'GG'" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#8, Column#8 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"Gg\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"GG-10\", ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", - " └─Projection 1.00 batchCop[tiflash] Column#10, cast(Column#11, var_string(20))->Column#13", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", - "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", - "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", - "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#9", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", - " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", - "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 batchCop[tiflash] Column#5", - " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#10, funcs:firstrow(Column#9)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#9 separator \",\")->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#11, Column#12, funcs:firstrow(Column#10)->Column#9", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#10, test.ts.col_1, test.ts.col_0", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", - "Plan": [ - "TableReader 8000.00 root data:ExchangeSender", - "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#19, funcs:group_concat(distinct Column#17 order by Column#18 separator \",\")->Column#5", - " └─Projection 8000.00 batchCop[tiflash] cast(Column#12, var_string(20))->Column#17, Column#13, test.ts.col_1", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#15, Column#16, funcs:firstrow(Column#14)->Column#13", - " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#14, test.ts.col_1, gt(cast(test.ts.col_0, double BINARY), 10)->Column#16", - " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" - ] - }, - { - "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts", - "Plan": [ - "HashAgg 1.00 root funcs:group_concat(distinct Column#6 order by Column#7 separator \",\")->Column#5", - "└─Projection 10000.00 root test.ts.col_0, nulleq(test.ts.col_0, )->Column#7", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tiflash] table:ts keep order:false, stats:pseudo" - ], - "Warning": [ - "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", - "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", - "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", - "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", - "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", - "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", - "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause" - ] - } - ] - }, - { - "Name": "TestRejectSortForMPP", - "Cases": [ - { - "SQL": "desc format = 'brief' select count(*) from (select * from t order by id)a group by name,id order by id", - "Plan": [ - "Projection 8000.00 root Column#5", - "└─Sort 8000.00 root test.t.id", - " └─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5, test.t.id", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: N/A], [name: test.t.id, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*) from (select * from t order by id)a group by name order by 1", - "Plan": [ - "Sort 8000.00 root Column#5", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#8", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select count(*) from (select id,name from t group by id,name order by id,name)a group by name order by 1", - "Plan": [ - "Sort 8000.00 root Column#5", - "└─TableReader 8000.00 root data:ExchangeSender", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 batchCop[tiflash] Column#5", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#5", - " └─Projection 8000.00 batchCop[tiflash] test.t.id, test.t.name", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.name)->test.t.name", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: N/A]", - " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select id from t group by id order by id)a join t on a.id=t.id order by 1", - "Plan": [ - "Sort 9990.00 root test.t.id", - "└─TableReader 9990.00 root data:ExchangeSender", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─Projection 9990.00 batchCop[tiflash] test.t.id, test.t.id, test.t.value, test.t.name", - " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", - " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", - " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: N/A]", - " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select * from t order by id)a join t on a.id=t.id order by 1", - "Plan": [ - "Sort 12487.50 root test.t.id", - "└─TableReader 12487.50 root data:ExchangeSender", - " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", - " └─Projection 12487.50 cop[tiflash] test.t.id, test.t.value, test.t.name, test.t.id, test.t.value, test.t.name", - " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", - " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from ((select id from t order by 1) union all (select id+1 from t order by 1))c", - "Plan": [ - "TableReader 20000.00 root data:ExchangeSender", - "└─ExchangeSender 20000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Union 20000.00 cop[tiflash] ", - " ├─Projection 10000.00 cop[tiflash] cast(test.t.id, bigint(20) BINARY)->Column#10", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] plus(test.t.id, 1)->Column#10", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from ((select count(*) from (select id,name from t order by id)a group by name,id order by id) union all (select id+1 from t order by 1))c", - "Plan": [ - "TableReader 18000.00 root data:ExchangeSender", - "└─ExchangeSender 18000.00 cop[tiflash] ExchangeType: PassThrough", - " └─Union 18000.00 cop[tiflash] ", - " ├─Projection 8000.00 cop[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", - " │ └─Projection 8000.00 cop[tiflash] Column#5", - " │ └─Projection 8000.00 cop[tiflash] Column#5, test.t.id", - " │ └─HashAgg 8000.00 cop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 8000.00 cop[tiflash] ", - " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: N/A], [name: test.t.id, collate: N/A]", - " │ └─HashAgg 8000.00 cop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", - " └─Projection 10000.00 cop[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", - " └─Projection 10000.00 cop[tiflash] plus(test.t.id, 1)->Column#11", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - }, - { - "SQL": "desc format = 'brief' select * from (select * from t order by id)a order by name", - "Plan": [ - "Sort 10000.00 root test.t.name", - "└─TableReader 10000.00 root data:ExchangeSender", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", - " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" - ] - } - ] - } -] diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 69d094c9b3c4c..b874166c28eaf 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -419,5 +419,453 @@ // Test correlated column in TablePath.TableFilters. "select c_int from tt1 where c_decimal > all (select /*+ use_index_merge(tt2) */ c_decimal from tt2 where tt2.c_int = 7 and tt2.c_int < tt1.c_decimal or tt2.c_str >= 'zzzzzzzzzzzzzzzzzzz' and tt1.c_int = tt2.c_int) order by 1;" ] + }, + { + "name": "TestIssue31240", + "cases": [ + "explain format = 'brief' select count(*) from t31240;", + "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", + "explain format = 'brief' select count(*) from t31240;" + ] + }, + { + "name": "TestSelPushDownTiFlash", + "cases": [ + "explain format = 'brief' select * from t where t.a > 1 and t.b = \"flash\" or t.a + 3 * t.a = 5", + "explain format = 'brief' select * from t where cast(t.a as double) + 3 = 5.1", + "explain format = 'brief' select * from t where b > 'a' order by convert(b, unsigned) limit 2", + "explain format = 'brief' select * from t where b > 'a' order by b limit 2" + ] + }, + { + "name": "TestVerboseExplain", + "cases": [ + "explain format = 'verbose' select count(*) from t3", + "explain format = 'verbose' select count(*) from t2", + "explain format = 'verbose' select * from t3 order by a", + "explain format = 'verbose' select * from t3 order by b", + "explain format = 'verbose' select * from t3 order by a limit 1", + "explain format = 'verbose' select * from t3 order by b limit 1", + "explain format = 'verbose' select count(*) from t2 group by a", + "explain format = 'verbose' select count(*) from t3 where b = 0", + "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", + "explain format = 'verbose' select count(*) from t2 where a = 0", + "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", + "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", + "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", + "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a" + ] + + }, + { + "name": "TestRegardNULLAsPoint", + "cases": [ + "select * from tuk where a<=>null and b=1", + "select * from tik where a<=>null and b=1", + "select * from tuk where a<=>null and b>0 and b<2", + "select * from tik where a<=>null and b>0 and b<2", + "select * from tuk where a<=>null and b>=1 and b<2", + "select * from tik where a<=>null and b>=1 and b<2", + "select * from tuk where a<=>null and b=1 and c=1", + "select * from tik where a<=>null and b=1 and c=1", + "select * from tuk where a=1 and b<=>null and c=1", + "select * from tik where a=1 and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c=1", + "select * from tik where a<=>null and b<=>null and c=1", + "select * from tuk where a<=>null and b<=>null and c<=>null", + "select * from tik where a<=>null and b<=>null and c<=>null" + ] + }, + { + "name": "TestPushDownToTiFlashWithKeepOrder", + "cases": [ + "explain format = 'brief' select max(a) from t", + "explain format = 'brief' select min(a) from t" + ] + }, + { + "name": "TestMPPJoin", + "cases": [ + "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", + "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" + ] + }, + { + "name": "TestMPPLeftSemiJoin", + "cases": [ + "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t); -- left semi", + "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t where b1 or t1.a not in (select a from test.t); -- left anti", + "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a not in (select a from test.t where b1 or t1.b in (select a from test.t); -- cartesian left semi", + "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select b from test.t where b1 or t1.b not in (select a from test.t); -- cartesian left anti", + "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.b not in (select a from test.t where b d1_t.value", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", + "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", + "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" + ] + }, + { + "name": "TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError", + "cases": [ + "explain format = 'brief' select v from t3 as a left join (select t1.v1, t1.v2, t1.v1 + t1.v2 as v from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2) b on a.v1 = b.v1 and a.v2 = b.v2", + "explain format = 'brief' select count(*), t2.v1, t2.v2 from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2 group by t2.v1, t2.v2", + "explain format = 'brief' select count(*), t2.v1, t2.v2 from t3 left join t2 on t3.v1 = t2.v1 and t3.v2 = t2.v2 group by t2.v1, t2.v2" + ] + }, + { + "name": "TestJoinNotSupportedByTiFlash", + "cases": [ + "explain format = 'brief' select * from table_1 a, table_1 b where a.bit_col = b.bit_col", + "explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100", + "explain format = 'brief' select * from table_1 a right join table_1 b on a.id = b.id and dayofmonth(b.datetime_col) > 100", + "explain format = 'brief' select * from table_1 a join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > dayofmonth(b.datetime_col)" + ] + }, + { + "name": "TestMPPWithHashExchangeUnderNewCollation", + "cases": [ + "explain format = 'brief' select * from table_1 a, table_1 b where a.value = b.value", + "explain format = 'brief' select * from table_1 a, table_2 b where a.value = b.value", + "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and b.value = c.value", + "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and a.value = c.value", + "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_1 group by value", + "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_2 group by value" + ] + }, + { + "name": "TestMPPWithBroadcastExchangeUnderNewCollation", + "cases": [ + "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.id = b.id", + "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.value = b.value" + ] + }, + { + "name": "TestMPPAvgRewrite", + "cases": [ + "explain format = 'brief' select /*+ avg_to_cop() */ id, avg(value+1),avg(value) from table_1 group by id" + ] + }, + { + "name": "TestReadFromStorageHint", + "cases": [ + "desc format = 'brief' select avg(a) from t", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a) from t", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a+1) from t", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(isnull(a)) from t", + "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIKV[t2]) */ * from t t1, t t2 where t1.a = t2.a", + "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIFLASH[t2]) */ * from t t1, t t2 where t1.a = t2.a", + "desc format = 'brief' select * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", + "desc format = 'brief' select /*+ read_from_storage(tiflash[tt]) */ * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", + "desc format = 'brief' select * from ttt order by ttt.a desc", + "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a desc", + "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a", + "desc format = 'brief' select /*+ read_from_storage(tikv[t, ttt]) */ * from ttt", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt" + ] + }, + { + "name": "TestReadFromStorageHintAndIsolationRead", + "cases": [ + "desc format = 'brief' select /*+ read_from_storage(tikv[t], tiflash[t]) */ avg(a) from t", + "desc format = 'brief' select /*+ read_from_storage(tikv[t]) */ avg(a) from t", + "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t" + ] + }, + { + "name": "TestIsolationReadDoNotFilterSystemDB", + "cases": [ + "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", + "desc format = 'brief' select * from information_schema.tables", + "desc format = 'brief' select * from mysql.stats_meta" + ] + }, + { + "name": "TestIsolationReadTiFlashNotChoosePointGet", + "cases": [ + "explain format = 'brief' select * from t where t.a = 1", + "explain format = 'brief' select * from t where t.a in (1, 2)" + ] + }, + { + "name": "TestIsolationReadTiFlashUseIndexHint", + "cases": [ + "explain format = 'brief' select * from t", + "explain format = 'brief' select * from t use index();", + "explain format = 'brief' select /*+ use_index(t, idx)*/ * from t", + "explain format = 'brief' select /*+ use_index(t)*/ * from t" + ] + }, + { + "name": "TestIssue20710", + "cases": [ + "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", + "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b" + ] + }, + { + "name": "TestPushDownProjectionForTiFlash", + "cases": [ + "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select * from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "desc format = 'brief' select A.id from t as A where exists (select 1 from t where t.id=A.id)", + "desc format = 'brief' select A.id from t as A where not exists (select 1 from t where t.id=A.id)", + "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;" + ] + }, + { + "name": "TestPushDownProjectionForMPP", + "cases": [ + "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select B.b+A.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", + "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "desc format = 'brief' select id from t as A where exists (select 1 from t where t.id=A.id)", + "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)", + "desc format = 'brief' select b*2, id from (select avg(value+2) as b, id from t group by id) C order by id", + "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;" + ] + }, + { + "name": "TestMppUnionAll", + "cases": [ + "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", + "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt" + ] + }, + { + "name": "TestMppJoinDecimal", + "cases": [ + "desc format = 'brief' select t1.c1, t1.c2, t2.c1, t2.c2, t2.c3 from t t1 join t t2 on t1.c1 + 1 = t2.c2 - 10 and t1.c1 * 3 = t2.c3 / 2", + "desc format = 'brief' select * from (select c1, c2, c5, count(*) c from t group by c1, c2, c5) t1 join (select c1, c2, c3, count(*) c from t group by c1, c2, c3) t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c5 = t2.c1", + "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c2 and t1.c3 = t2.c3 and t1.c4 = t2.c4 and t1.c5 = t2.c5", + "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", + "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", + "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", + "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", + "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2" + ] + }, + { + "name": "TestPushDownAggForMPP", + "cases": [ + "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id+1 from t)A", + "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "desc format = 'brief' select count(*) from t", + "desc format = 'brief' select count(*), id from t group by id", + "desc format = 'brief' select count(*), id + 1 from t group by id + 1", + "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", + "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", + "desc format = 'brief' select avg(value) as b,id from t group by id", + "desc format = 'brief' select /*+hash_agg()*/ sum(b) from (select avg(value) as b, id from t group by id)A", + "desc format = 'brief' select id from t group by id having avg(value)>0", + "desc format = 'brief' select avg(value),id from t group by id having avg(value)>0", + "desc format = 'brief' select avg(value) +1,id from t group by id", + "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", + "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "desc format = 'brief' select count(distinct value),id from t group by id", + "desc format = 'brief' select count(distinct value),sum(distinct value),id from t group by id", + "desc format = 'brief' select * from t join ( select count(distinct value), id from t group by id) as A on A.id = t.id", + "desc format = 'brief' select * from t join ( select count(1/value), id from t group by id) as A on A.id = t.id", + "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", + "desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/", + "desc format = 'brief' select count(distinct value) from t", + "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", + "desc format = 'brief' select count(distinct value), count(value), avg(value) from t" + ] + }, + { + "name": "TestMppAggTopNWithJoin", + "cases": [ + "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", + "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", + "desc format = 'brief' select * from t join ( select count(*) as v, id from t group by value,id having value+v <10) as A on A.id = t.id", + "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", + "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", + "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", + "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1" + ] + }, + { + "name": "TestIndexMergeSerial", + "cases": [ + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)" + ] + }, + { + "name": "TestLimitIndexLookUpKeepOrder", + "cases": [ + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10" + ] + }, + { + "name": "TestIssue23887", + "cases": [ + "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" + ] + }, + { + "name": "TestMergeContinuousSelections", + "cases": [ + "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;" + ] + }, + { + "name": "TestPushDownGroupConcatToTiFlash", + "cases": [ + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", + "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts" + ] + }, + { + "name": "TestRejectSortForMPP", + "cases": [ + "desc format = 'brief' select count(*) from (select * from t order by id)a group by name,id order by id", + "desc format = 'brief' select count(*) from (select * from t order by id)a group by name order by 1", + "desc format = 'brief' select count(*) from (select id,name from t group by id,name order by id,name)a group by name order by 1", + "desc format = 'brief' select * from (select id from t group by id order by id)a join t on a.id=t.id order by 1", + "desc format = 'brief' select * from (select * from t order by id)a join t on a.id=t.id order by 1", + "desc format = 'brief' select * from ((select id from t order by 1) union all (select id+1 from t order by 1))c", + "desc format = 'brief' select * from ((select count(*) from (select id,name from t order by id)a group by name,id order by id) union all (select id+1 from t order by 1))c", + "desc format = 'brief' select * from (select * from t order by id)a order by name" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index dd4330fe1310e..a1ab58e448d04 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -2062,12 +2062,12 @@ { "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", "Plan": [ - "Projection 15.99 root 1->Column#5", - "└─Selection 15.99 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", - " └─IndexMerge 19.99 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" + "Projection 17.99 root 1->Column#5", + "└─IndexMerge 0.04 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.04 cop[tikv] or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", + " └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Res": [ "1" @@ -2077,11 +2077,11 @@ "SQL": "select /*+ use_index_merge(t1) */ 1 from t1 where c1 = 'ab' or c2 = '10' and char_length(left(c1, 10)) = 10;", "Plan": [ "Projection 17.99 root 1->Column#5", - "└─Selection 0.04 root or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.t1.c1, 10)), 10)))", - " └─IndexMerge 19.99 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"ab\",\"ab\"], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" + "└─IndexMerge 0.04 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"ab\",\"ab\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.04 cop[tikv] or(eq(test.t1.c1, \"ab\"), and(eq(test.t1.c2, \"10\"), eq(char_length(left(test.t1.c1, 10)), 10)))", + " └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Res": [ "1" @@ -2090,12 +2090,12 @@ { "SQL": "select /*+ use_index_merge(tt1) */ 1 from tt1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c3)) = '10';", "Plan": [ - "Projection 15.99 root 1->Column#6", - "└─Selection 15.99 root or(eq(test.tt1.c1, \"de\"), and(eq(test.tt1.c2, \"10\"), eq(from_base64(to_base64(test.tt1.c3)), \"10\")))", - " └─IndexMerge 19.99 root ", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", - " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 19.99 cop[tikv] table:tt1 keep order:false, stats:pseudo" + "Projection 17.99 root 1->Column#6", + "└─IndexMerge 17.99 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:tt1, index:idx_0(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", + " ├─Selection(Build) 8.00 cop[tikv] eq(from_base64(to_base64(test.tt1.c3)), \"10\")", + " │ └─IndexRangeScan 10.00 cop[tikv] table:tt1, index:idx_1(c2, c3) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 17.99 cop[tikv] table:tt1 keep order:false, stats:pseudo" ], "Res": [ "1" @@ -2104,12 +2104,13 @@ { "SQL": "select /*+ use_index_merge( tt2 ) */ 1 from tt2 where tt2.c1 in (-3896405) or tt2.pk in (1, 53330) and to_base64(left(pk, 5));", "Plan": [ - "Projection 2.40 root 1->Column#3", - "└─Selection 2.40 root or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))))", - " └─IndexMerge 3.00 root ", - " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", - " ├─TableRangeScan(Build) 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo", - " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" + "Projection 2.60 root 1->Column#3", + "└─IndexMerge 0.00 root ", + " ├─IndexRangeScan(Build) 1.00 cop[tikv] table:tt2, index:c1(c1) range:[-3896405,-3896405], keep order:false, stats:pseudo", + " ├─Selection(Build) 1.60 cop[tikv] istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))", + " │ └─TableRangeScan 2.00 cop[tikv] table:tt2 range:[1,1], [53330,53330], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] or(eq(test.tt2.c1, -3896405), and(in(test.tt2.pk, 1, 53330), istrue_with_null(cast(to_base64(left(cast(test.tt2.pk, var_string(20)), 5)), double BINARY))))", + " └─TableRowIDScan 3.00 cop[tikv] table:tt2 keep order:false, stats:pseudo" ], "Res": [ "1" @@ -2132,14 +2133,16 @@ { "SQL": "select 1 from t1 where c1 = 'de' or c2 = '10' and from_base64(to_base64(c1)) = 'ab';", "Plan": [ - "Projection 8000.00 root 1->Column#5", - "└─Selection 8000.00 root or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "Projection 17.99 root 1->Column#5", + "└─IndexMerge 0.04 root ", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c1(c1) range:[\"de\",\"de\"], keep order:false, stats:pseudo", + " ├─IndexRangeScan(Build) 10.00 cop[tikv] table:t1, index:c2(c2) range:[\"10\",\"10\"], keep order:false, stats:pseudo", + " └─Selection(Probe) 0.04 cop[tikv] or(eq(test.t1.c1, \"de\"), and(eq(test.t1.c2, \"10\"), eq(from_base64(to_base64(test.t1.c1)), \"ab\")))", + " └─TableRowIDScan 19.99 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Res": [ "1" - ] + ] } ] }, @@ -2259,5 +2262,4305 @@ ] } ] + }, + { + "Name": "TestIssue31240", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set @@tidb_isolation_read_engines=\"tiflash,tidb\";", + "Plan": null + }, + { + "SQL": "explain format = 'brief' select count(*) from t31240;", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t31240 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestSelPushDownTiFlash", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where t.a > 1 and t.b = \"flash\" or t.a + 3 * t.a = 5", + "Plan": [ + "TableReader 8000.67 root data:Selection", + "└─Selection 8000.67 cop[tiflash] or(and(gt(test.t.a, 1), eq(test.t.b, \"flash\")), eq(plus(test.t.a, mul(3, test.t.a)), 5))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where cast(t.a as double) + 3 = 5.1", + "Plan": [ + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tiflash] eq(plus(cast(test.t.a, double BINARY), 3), 5.1)", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b > 'a' order by convert(b, unsigned) limit 2", + "Plan": [ + "Projection 2.00 root test.t.a, test.t.b", + "└─TopN 2.00 root Column#4, offset:0, count:2", + " └─Projection 2.00 root test.t.a, test.t.b, cast(test.t.b, bigint(22) UNSIGNED BINARY)->Column#4", + " └─TableReader 2.00 root data:Projection", + " └─Projection 2.00 batchCop[tiflash] test.t.a, test.t.b", + " └─TopN 2.00 batchCop[tiflash] Column#3, offset:0, count:2", + " └─Projection 3333.33 batchCop[tiflash] test.t.a, test.t.b, cast(test.t.b, bigint(22) UNSIGNED BINARY)->Column#3", + " └─Selection 3333.33 batchCop[tiflash] gt(test.t.b, \"a\")", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where b > 'a' order by b limit 2", + "Plan": [ + "TopN 2.00 root test.t.b, offset:0, count:2", + "└─TableReader 2.00 root data:TopN", + " └─TopN 2.00 batchCop[tiflash] test.t.b, offset:0, count:2", + " └─Selection 3333.33 batchCop[tiflash] gt(test.t.b, \"a\")", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestVerboseExplain", + "Cases": [ + { + "SQL": "explain format = 'verbose' select count(*) from t3", + "Plan": [ + "StreamAgg_20 1.00 137.00 root funcs:count(Column#9)->Column#4", + "└─TableReader_21 1.00 9.68 root data:StreamAgg_8", + " └─StreamAgg_8 1.00 12.68 cop[tikv] funcs:count(1)->Column#9", + " └─TableFullScan_18 3.00 128.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2", + "Plan": [ + "StreamAgg_25 1.00 69.50 root funcs:count(Column#7)->Column#4", + "└─TableReader_26 1.00 5.17 root data:StreamAgg_9", + " └─StreamAgg_9 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#7", + " └─TableFullScan_24 3.00 60.50 batchCop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a", + "Plan": [ + "Sort_4 3.00 45.85 root test.t3.a", + "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 128.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b", + "Plan": [ + "Sort_4 3.00 45.85 root test.t3.b", + "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 128.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", + "Plan": [ + "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 10.22 root data:TopN_15", + " └─TopN_15 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", + "Plan": [ + "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 10.22 root data:TopN_15", + " └─TopN_15 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 128.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 group by a", + "Plan": [ + "TableReader_24 3.00 3.33 root data:ExchangeSender_23", + "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", + " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_21 3.00 68.00 batchCop[tiflash] ", + " └─ExchangeSender_20 3.00 68.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─TableFullScan_19 3.00 65.00 batchCop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 1.33 root funcs:count(1)->Column#4", + "└─IndexReader_15 0.00 1.33 root index:IndexRangeScan_14", + " └─IndexRangeScan_14 0.00 20.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", + "Plan": [ + "StreamAgg_10 1.00 19.33 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 19.33 root ", + " ├─IndexRangeScan_15(Build) 0.00 20.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", + " └─TableRowIDScan_16(Probe) 0.00 20.00 cop[tikv] table:t3 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", + "Plan": [ + "StreamAgg_11 1.00 4.93 root funcs:count(1)->Column#4", + "└─TableReader_23 0.00 4.93 root data:Selection_22", + " └─Selection_22 0.00 74.00 cop[tiflash] eq(test.t2.a, 0)", + " └─TableFullScan_21 3.00 65.00 cop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", + "Plan": [ + "StreamAgg_10 1.00 60.22 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 51.22 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 11.66 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 10.76 root data:Selection_25", + " └─Selection_25 3.00 137.00 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 128.00 cop[tikv] table:t keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_12 1.00 18.93 root funcs:count(1)->Column#7", + "└─TableReader_45 3.00 9.93 root data:ExchangeSender_44", + " └─ExchangeSender_44 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin_41 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_20(Build) 3.00 77.00 cop[tiflash] ", + " │ └─ExchangeSender_19 3.00 77.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection_18 3.00 74.00 cop[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_17 3.00 65.00 cop[tiflash] table:t1 keep order:false", + " └─Selection_22(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_21 3.00 65.00 cop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", + "Plan": [ + "StreamAgg_15 1.00 60.60 root funcs:count(1)->Column#10", + "└─HashJoin_65 3.00 51.60 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", + " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", + " │ └─ExchangeSender_34 3.00 106.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection_33 3.00 103.00 cop[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 94.00 cop[tiflash] table:t1 keep order:false", + " └─Selection_37(Probe) 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 65.00 cop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", + "Plan": [ + "HashJoin_19 3.00 127.40 root CARTESIAN left outer semi join", + "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", + "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", + "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", + "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 95.82 root 1->Column#28", + " └─Apply_22 3.00 76.02 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 1.00 21.95 root 1->Column#26", + " └─Limit_30 1.00 3.35 root offset:0, count:1", + " └─TableReader_38 1.00 3.35 root data:ExchangeSender_37", + " └─ExchangeSender_37 1.00 79.50 cop[tiflash] ExchangeType: PassThrough", + " └─Limit_36 1.00 79.50 cop[tiflash] offset:0, count:1", + " └─TableFullScan_35 1.00 79.50 cop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'verbose' select /*+ merge_join(t1) */ count(*) from t1 join t2 on t1.a = t2.a", + "Plan": [ + "StreamAgg_11 1.00 59.65 root funcs:count(1)->Column#7", + "└─MergeJoin_30 3.00 50.65 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_28(Build) 3.00 20.83 root test.t2.a", + " │ └─TableReader_27 3.00 6.56 root data:Selection_26", + " │ └─Selection_26 3.00 74.00 cop[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_25 3.00 65.00 cop[tiflash] table:t2 keep order:false", + " └─Sort_21(Probe) 3.00 20.83 root test.t1.a", + " └─TableReader_20 3.00 6.56 root data:Selection_19", + " └─Selection_19 3.00 74.00 cop[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_18 3.00 65.00 cop[tiflash] table:t1 keep order:false" + ] + } + ] + }, + { + "Name": "TestRegardNULLAsPoint", + "Cases": [ + { + "SQL": "select * from tuk where a<=>null and b=1", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.01 root index:Selection_6", + "└─Selection_6 0.01 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>0 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tuk.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b>=1 and b<2", + "PlanEnabled": [ + "IndexReader_6 0.10 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1,NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.25 root index:Selection_6", + "└─Selection_6 0.25 cop[tikv] eq(test.tik.b, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 ", + " 1 ", + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.b, 1), eq(test.tuk.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b=1 and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL 1 1,NULL 1 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.b, 1), eq(test.tik.c, 1)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1 1", + " 1 1" + ] + }, + { + "SQL": "select * from tuk where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1)", + " └─IndexRangeScan_5 0.10 cop[tikv] table:tuk, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tik where a=1 and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL 1,1 NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1)", + " └─IndexRangeScan_5 0.10 cop[tikv] table:tik, index:a(a, b, c) range:[1 NULL,1 NULL], keep order:false, stats:pseudo" + ], + "Result": [ + "1 1", + "1 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tuk.c, 1), nulleq(test.tuk.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c=1", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL 1,NULL NULL 1], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] eq(test.tik.c, 1), nulleq(test.tik.b, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " 1", + " 1" + ] + }, + { + "SQL": "select * from tuk where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "IndexReader_6 1.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 1.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tuk.b, NULL), nulleq(test.tuk.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tuk, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + }, + { + "SQL": "select * from tik where a<=>null and b<=>null and c<=>null", + "PlanEnabled": [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL NULL NULL,NULL NULL NULL], keep order:false, stats:pseudo" + ], + "PlanDisabled": [ + "IndexReader_7 0.00 root index:Selection_6", + "└─Selection_6 0.00 cop[tikv] nulleq(test.tik.b, NULL), nulleq(test.tik.c, NULL)", + " └─IndexRangeScan_5 10.00 cop[tikv] table:tik, index:a(a, b, c) range:[NULL,NULL], keep order:false, stats:pseudo" + ], + "Result": [ + " ", + " " + ] + } + ] + }, + { + "Name": "TestPushDownToTiFlashWithKeepOrder", + "Cases": [ + { + "SQL": "explain format = 'brief' select max(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:max(test.t.a)->Column#3", + "└─TopN 1.00 root test.t.a:desc, offset:0, count:1", + " └─TableReader 1.00 root data:TopN", + " └─TopN 1.00 batchCop[tiflash] test.t.a:desc, offset:0, count:1", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select min(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:min(test.t.a)->Column#3", + "└─Limit 1.00 root offset:0, count:1", + " └─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tiflash] offset:0, count:1", + " └─TableFullScan 1.00 cop[tiflash] table:t keep order:true, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMPPJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#17", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", + " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", + " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", + " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 cop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 8.00 cop[tiflash] ", + " │ └─ExchangeSender 8.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 cop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPLeftSemiJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t); -- left semi", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 cop[tiflash] left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select a from test.t where b1 or t1.a not in (select a from test.t); -- left anti", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 cop[tiflash] anti left outer semi join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a not in (select a from test.t where b1 or t1.b in (select a from test.t); -- cartesian left semi", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 cop[tiflash] CARTESIAN left outer semi join, other cond:eq(test.t.b, test.t.a)", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.a in (select b from test.t where b1 or t1.b not in (select a from test.t); -- cartesian left anti", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 cop[tiflash] test.t.a, test.t.b", + " └─Selection 8000.00 cop[tiflash] or(gt(test.t.a, 1), Column#7)", + " └─HashJoin 10000.00 cop[tiflash] CARTESIAN anti left outer semi join, other cond:eq(test.t.b, test.t.a)", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select * from test.t t1 where t1.a>1 or t1.b not in (select a from test.t where bColumn#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.a.id, collate: binary]", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.b.id, collate: binary]", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPShuffledJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#18)->Column#17", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", + " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", + " └─ExchangeReceiver(Probe) 64.00 batchCop[tiflash] ", + " └─ExchangeSender 64.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: binary]", + " └─HashJoin 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", + " └─ExchangeReceiver(Probe) 32.00 batchCop[tiflash] ", + " └─ExchangeSender 32.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: binary]", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d1_k = d2_t.value and fact_t.d1_k = d3_t.value", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#18)->Column#17", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", + " └─HashJoin 128.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d3_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 64.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d2_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#22)->Column#19", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#22", + " └─HashJoin 204.80 batchCop[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Projection(Probe) 102.40 batchCop[tiflash] test.fact_t.d1_k", + " └─Selection 102.40 batchCop[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", + " └─HashJoin 128.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", + " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 16.00 batchCop[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", + " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", + " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", + " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#13)->Column#12", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─HashJoin 12.80 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#13)->Column#12", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─HashJoin 12.80 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#13)->Column#12", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─HashJoin 12.80 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#13)->Column#12", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", + " └─HashJoin 12.80 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPJoinWithCanNotFoundColumnInSchemaColumnsError", + "Cases": [ + { + "SQL": "explain format = 'brief' select v from t3 as a left join (select t1.v1, t1.v2, t1.v1 + t1.v2 as v from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2) b on a.v1 = b.v1 and a.v2 = b.v2", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 cop[tiflash] Column#13", + " └─HashJoin 1.00 cop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t1.v1) eq(test.t3.v2, test.t1.v2)]", + " ├─ExchangeReceiver(Build) 1.00 cop[tiflash] ", + " │ └─ExchangeSender 1.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#23, collate: binary], [name: Column#24, collate: binary]", + " │ └─Projection 1.00 cop[tiflash] test.t3.v1, test.t3.v2, cast(test.t3.v1, decimal(20,2))->Column#23, cast(test.t3.v2, decimal(20,2))->Column#24", + " │ └─TableFullScan 1.00 cop[tiflash] table:a keep order:false", + " └─Projection(Probe) 2.00 cop[tiflash] test.t1.v1, test.t1.v2, plus(test.t1.v1, test.t1.v2)->Column#13", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.t1.v1)), not(isnull(test.t1.v2))", + " │ └─TableFullScan 2.00 cop[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 cop[tiflash] ", + " └─ExchangeSender 8.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: binary], [name: Column#16, collate: binary]", + " └─Projection 8.00 cop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#15, cast(test.t2.v2, decimal(20,2))->Column#16", + " └─Selection 8.00 cop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 cop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t1 left join t2 on t1.v1 = t2.v1 and t1.v2 = t2.v2 group by t2.v1, t2.v2", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#22)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─ExchangeReceiver 2.00 batchCop[tiflash] ", + " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#22", + " └─HashJoin 2.00 batchCop[tiflash] left outer join, equal:[eq(test.t1.v1, test.t2.v1) eq(test.t1.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.v1, collate: binary], [name: test.t1.v2, collate: binary]", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", + " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary], [name: Column#15, collate: binary]", + " └─Projection 8.00 batchCop[tiflash] test.t2.v1, test.t2.v2, cast(test.t2.v1, decimal(20,2))->Column#14, cast(test.t2.v2, decimal(20,2))->Column#15", + " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*), t2.v1, t2.v2 from t3 left join t2 on t3.v1 = t2.v1 and t3.v2 = t2.v2 group by t2.v1, t2.v2", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#9, test.t2.v1, test.t2.v2", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:sum(Column#16)->Column#9, funcs:firstrow(test.t2.v1)->test.t2.v1, funcs:firstrow(test.t2.v2)->test.t2.v2", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t2.v1, test.t2.v2, funcs:count(1)->Column#16", + " └─HashJoin 1.00 batchCop[tiflash] left outer join, equal:[eq(test.t3.v1, test.t2.v1) eq(test.t3.v2, test.t2.v2)]", + " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t3.v1, collate: binary], [name: test.t3.v2, collate: binary]", + " │ └─TableFullScan 1.00 batchCop[tiflash] table:t3 keep order:false", + " └─ExchangeReceiver(Probe) 8.00 batchCop[tiflash] ", + " └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.v1, collate: binary], [name: test.t2.v2, collate: binary]", + " └─Selection 8.00 batchCop[tiflash] not(isnull(test.t2.v1)), not(isnull(test.t2.v2))", + " └─TableFullScan 8.00 batchCop[tiflash] table:t2 keep order:false" + ] + } + ] + }, + { + "Name": "TestJoinNotSupportedByTiFlash", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from table_1 a, table_1 b where a.bit_col = b.bit_col", + "Plan": [ + "HashJoin 2.00 root inner join, equal:[eq(test.table_1.bit_col, test.table_1.bit_col)]", + "├─TableReader(Build) 2.00 root data:TableFullScan", + "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", + "└─TableReader(Probe) 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a left join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > 100", + "Plan": [ + "HashJoin 2.00 root left outer join, equal:[eq(test.table_1.id, test.table_1.id)], left cond:[gt(dayofmonth(test.table_1.datetime_col), 100)]", + "├─TableReader(Build) 2.00 root data:TableFullScan", + "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", + "└─TableReader(Probe) 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a right join table_1 b on a.id = b.id and dayofmonth(b.datetime_col) > 100", + "Plan": [ + "HashJoin 2.00 root right outer join, equal:[eq(test.table_1.id, test.table_1.id)], right cond:gt(dayofmonth(test.table_1.datetime_col), 100)", + "├─TableReader(Build) 2.00 root data:TableFullScan", + "│ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + "└─TableReader(Probe) 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a join table_1 b on a.id = b.id and dayofmonth(a.datetime_col) > dayofmonth(b.datetime_col)", + "Plan": [ + "HashJoin 2.00 root inner join, equal:[eq(test.table_1.id, test.table_1.id)], other cond:gt(dayofmonth(test.table_1.datetime_col), dayofmonth(test.table_1.datetime_col))", + "├─TableReader(Build) 2.00 root data:TableFullScan", + "│ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", + "└─TableReader(Probe) 2.00 root data:TableFullScan", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPWithHashExchangeUnderNewCollation", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from table_1 a, table_1 b where a.value = b.value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a, table_2 b where a.value = b.value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", + " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and b.value = c.value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_2.value, test.table_1.value)]", + " ├─HashJoin(Build) 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select * from table_1 a, table_2 b, table_1 c where a.value = b.value and a.value = c.value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " │ └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_2.value)]", + " │ ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_bin]", + " │ │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " │ │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " │ └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_2.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 cop[tiflash] table:c keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_1 group by value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_1.value", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_1.value)->test.table_1.value", + " └─ExchangeReceiver 2.00 batchCop[tiflash] ", + " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.value, collate: utf8mb4_general_ci]", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.value, funcs:count(1)->Column#7", + " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ agg_to_cop() */ count(*), value from table_2 group by value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 batchCop[tiflash] Column#4, test.table_2.value", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.table_2.value)->test.table_2.value", + " └─ExchangeReceiver 2.00 batchCop[tiflash] ", + " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_2.value, collate: utf8mb4_bin]", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_2.value, funcs:count(1)->Column#7", + " └─TableFullScan 2.00 batchCop[tiflash] table:table_2 keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPWithBroadcastExchangeUnderNewCollation", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.id = b.id", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.id, test.table_1.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ broadcast_join(a,b) */ * from table_1 a, table_1 b where a.value = b.value", + "Plan": [ + "TableReader 2.00 root data:ExchangeSender", + "└─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] inner join, equal:[eq(test.table_1.value, test.table_1.value)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─Selection(Probe) 2.00 cop[tiflash] not(isnull(test.table_1.value))", + " └─TableFullScan 2.00 cop[tiflash] table:b keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPAvgRewrite", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ avg_to_cop() */ id, avg(value+1),avg(value) from table_1 group by id", + "Plan": [ + "Projection 2.00 root test.table_1.id, Column#4, Column#5", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 2.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#4, div(Column#5, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#5, test.table_1.id", + " └─HashAgg 2.00 batchCop[tiflash] group by:test.table_1.id, funcs:sum(Column#15)->Column#13, funcs:sum(Column#16)->Column#4, funcs:sum(Column#17)->Column#14, funcs:sum(Column#18)->Column#5, funcs:firstrow(test.table_1.id)->test.table_1.id", + " └─ExchangeReceiver 2.00 batchCop[tiflash] ", + " └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.table_1.id, collate: binary]", + " └─HashAgg 2.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#25)->Column#15, funcs:sum(Column#26)->Column#16, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#18", + " └─Projection 2.00 batchCop[tiflash] plus(test.table_1.value, 1)->Column#25, plus(test.table_1.value, 1)->Column#26, test.table_1.value, test.table_1.value, test.table_1.id", + " └─TableFullScan 2.00 batchCop[tiflash] table:table_1 keep order:false" + ] + } + ] + }, + { + "Name": "TestReadFromStorageHint", + "Cases": [ + { + "SQL": "desc format = 'brief' select avg(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(14,4) BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", + " └─Projection 10000.00 batchCop[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(a+1) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.a, 1), decimal(20,0) BINARY)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ sum(isnull(a)) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(Column#6)->Column#4", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#7)->Column#6", + " └─Projection 10000.00 batchCop[tiflash] cast(isnull(test.t.a), decimal(20,0) BINARY)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIKV[t2]) */ * from t t1, t t2 where t1.a = t2.a", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ READ_FROM_STORAGE(TIKV[t1], TIFLASH[t2]) */ * from t t1, t t2 where t1.a = t2.a", + "Plan": [ + "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", + "Plan": [ + "TableReader 44.00 root data:TableRangeScan", + "└─TableRangeScan 44.00 cop[tiflash] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[tt]) */ * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", + "Plan": [ + "TableReader 44.00 root data:TableRangeScan", + "└─TableRangeScan 44.00 cop[tiflash] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select * from ttt order by ttt.a desc", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:ttt keep order:true, desc, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a desc", + "Plan": [ + "Sort 10000.00 root test.ttt.a:desc", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[ttt]) */ * from ttt order by ttt.a", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:true, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t, ttt]) */ * from ttt", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:ttt keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ READ_FROM_STORAGE(tikv[t, ttt]) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t, ttt], tikv[tt]) */ * from ttt", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:ttt keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t, tt) in optimizer hint /*+ READ_FROM_STORAGE(tiflash[t, ttt], tikv[tt]) */. Maybe you can use the table alias name" + ] + } + ] + }, + { + "Name": "TestReadFromStorageHintAndIsolationRead", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t], tiflash[t]) */ avg(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Storage hints are conflict, you can only specify one storage type of table test.t" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tikv[t]) */ avg(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "desc format = 'brief' select /*+ read_from_storage(tiflash[t]) */ avg(a) from t", + "Plan": [ + "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", + "└─IndexReader 1.00 root index:StreamAgg", + " └─StreamAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#7, funcs:sum(test.t.a)->Column#8", + " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{}])" + ] + } + ] + }, + { + "Name": "TestIsolationReadDoNotFilterSystemDB", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from metrics_schema.tidb_query_duration where time >= '2019-12-23 16:10:13' and time <= '2019-12-23 16:30:13'", + "Plan": [ + "MemTableScan 10000.00 root table:tidb_query_duration PromQL:histogram_quantile(0.9, sum(rate(tidb_server_handle_query_duration_seconds_bucket{}[60s])) by (le,sql_type,instance)), start_time:2019-12-23 16:10:13, end_time:2019-12-23 16:30:13, step:1m0s" + ] + }, + { + "SQL": "desc format = 'brief' select * from information_schema.tables", + "Plan": [ + "MemTableScan 10000.00 root table:TABLES " + ] + }, + { + "SQL": "desc format = 'brief' select * from mysql.stats_meta", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:stats_meta keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIsolationReadTiFlashNotChoosePointGet", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t where t.a = 1", + "Result": [ + "TableReader 1.00 root data:TableRangeScan", + "└─TableRangeScan 1.00 cop[tiflash] table:t range:[1,1], keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where t.a in (1, 2)", + "Result": [ + "TableReader 2.00 root data:TableRangeScan", + "└─TableRangeScan 2.00 cop[tiflash] table:t range:[1,1], [2,2], keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIsolationReadTiFlashUseIndexHint", + "Cases": [ + { + "SQL": "explain format = 'brief' select * from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select * from t use index();", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "explain format = 'brief' select /*+ use_index(t, idx)*/ * from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "TiDB doesn't support index in the isolation read engines(value: 'tiflash')" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ use_index(t)*/ * from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] + }, + { + "Name": "TestIssue20710", + "Cases": [ + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_join(s) */ * from t join s on t.a=s.a and t.a = s.b", + "Plan": [ + "IndexJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.b", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.b)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.b = s.a", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.b, test.s.a)", + "├─TableReader(Build) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select /*+ inl_hash_join(s) */ * from t join s on t.a=s.a and t.a = s.b", + "Plan": [ + "IndexHashJoin 12475.01 root inner join, inner:IndexLookUp, outer key:test.t.a, inner key:test.s.a, equal cond:eq(test.t.a, test.s.a), eq(test.t.a, test.s.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test.s.a))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:s, index:a(a) range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo", + " └─Selection(Probe) 1.25 cop[tikv] not(isnull(test.s.b))", + " └─TableRowIDScan 1.25 cop[tikv] table:s keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownProjectionForTiFlash", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:HashAgg", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#7)->Column#6", + "└─TableReader 1.00 root data:HashAgg", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#8)->Column#6", + "└─TableReader 1.00 root data:HashAgg", + " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", + "Plan": [ + "StreamAgg 1.00 root funcs:count(Column#7)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", + "Plan": [ + "StreamAgg 1.00 root funcs:sum(Column#8)->Column#6", + "└─TableReader 1.00 root data:StreamAgg", + " └─StreamAgg 1.00 batchCop[tiflash] funcs:sum(Column#10)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "Plan": [ + "TableReader 10000.00 root data:HashJoin", + "└─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 10000.00 root data:HashJoin", + "└─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", + " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 10000.00 root data:HashJoin", + "└─HashJoin 10000.00 cop[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", + " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 12487.50 root data:HashJoin", + "└─HashJoin 12487.50 cop[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", + " ├─Selection(Build) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─Projection(Probe) 10000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "Plan": [ + "Projection 10000.00 root Column#10, Column#5", + "└─TableReader 10000.00 root data:HashJoin", + " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─Projection(Build) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select A.id from t as A where exists (select 1 from t where t.id=A.id)", + "Plan": [ + "TableReader 7992.00 root data:HashJoin", + "└─HashJoin 7992.00 cop[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─Selection(Build) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select A.id from t as A where not exists (select 1 from t where t.id=A.id)", + "Plan": [ + "TableReader 8000.00 root data:HashJoin", + "└─HashJoin 8000.00 cop[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─TableFullScan(Build) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo, global read", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;", + "Plan": [ + "Projection 10000.00 root from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownProjectionForMPP", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#9)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#8)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#9)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#10)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#11)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ count(*) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#9)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#9", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ stream_agg()*/ sum(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#10)->Column#6", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#11)->Column#10", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select B.b+A.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 cop[tiflash] plus(Column#5, Column#10)->Column#11", + " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", + " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", + " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t left join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 cop[tiflash] left outer join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", + " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] right outer join, equal:[eq(test.t.id, Column#9)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 10000.00 cop[tiflash] minus(test.t.id, 2)->Column#9", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select A.b, B.b from (select id-2 as b from t) B join (select id-2 as b from t) A on A.b=B.b", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 cop[tiflash] Column#10, Column#5", + " └─HashJoin 10000.00 cop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 8000.00 cop[tiflash] ", + " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#5", + " │ └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 8000.00 cop[tiflash] minus(test.t.id, 2)->Column#10", + " └─Selection 8000.00 cop[tiflash] not(isnull(minus(test.t.id, 2)))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select id from t as A where exists (select 1 from t where t.id=A.id)", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 cop[tiflash] semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 cop[tiflash] anti semi join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 cop[tiflash] table:A keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select b*2, id from (select avg(value+2) as b, id from t group by id) C order by id", + "Plan": [ + "Sort 8000.00 root test.t.id", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] mul(Column#5, 2)->Column#6, test.t.id", + " └─Projection 8000.00 batchCop[tiflash] div(Column#5, cast(case(eq(Column#20, 0), 1, Column#20), decimal(20,0) BINARY))->Column#5, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#21)->Column#20, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#26, funcs:count(Column#24)->Column#21, funcs:sum(Column#25)->Column#22", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.value, 2)->Column#24, plus(test.t.value, 2)->Column#25, test.t.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' SELECT FROM_UNIXTIME(name,'%Y-%m-%d') FROM t;", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 10000.00 cop[tiflash] from_unixtime(cast(test.t.name, decimal(65,0) BINARY), %Y-%m-%d)->Column#5", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMppUnionAll", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#12", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#9, test.t.b", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#10", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#16)->Column#15", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#16", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#13, test.t.b", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#14", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#14", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " │ └─Projection 10000.00 cop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.c, double BINARY)->Column#10", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#14", + " └─Union 10000.00 batchCop[tiflash] ", + " └─Projection 10000.00 batchCop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─Union 0.00 root ", + " ├─Projection 0.00 root test.t.a, cast(test.t.b, double BINARY)->Column#10", + " │ └─TableDual 0.00 root rows:0", + " └─Projection 0.00 root test.t1.a, cast(test.t1.c, double BINARY)->Column#10", + " └─TableDual 0.00 root rows:0" + ] + } + ] + }, + { + "Name": "TestMppJoinDecimal", + "Cases": [ + { + "SQL": "desc format = 'brief' select t1.c1, t1.c2, t2.c1, t2.c2, t2.c3 from t t1 join t t2 on t1.c1 + 1 = t2.c2 - 10 and t1.c1 * 3 = t2.c3 / 2", + "Plan": [ + "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c1, test.t.c2, test.t.c3", + "└─TableReader 12500.00 root data:ExchangeSender", + " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14) eq(Column#15, Column#16)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#21, collate: binary], [name: Column#15, collate: binary]", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, Column#13, Column#15, cast(Column#13, decimal(34,8) BINARY)->Column#21", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, mul(test.t.c1, 3)->Column#13, plus(test.t.c1, 1)->Column#15", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#22, collate: binary], [name: Column#16, collate: binary]", + " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, Column#14, Column#16, cast(Column#14, decimal(34,8) BINARY)->Column#22", + " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, div(test.t.c3, 2)->Column#14, minus(test.t.c2, 10)->Column#16", + " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select c1, c2, c5, count(*) c from t group by c1, c2, c5) t1 join (select c1, c2, c3, count(*) c from t group by c1, c2, c3) t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c5 = t2.c1", + "Plan": [ + "Projection 7976.02 root test.t.c1, test.t.c2, test.t.c5, Column#7, test.t.c1, test.t.c2, test.t.c3, Column#14", + "└─TableReader 7976.02 root data:ExchangeSender", + " └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7976.02 batchCop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c5, test.t.c1)]", + " ├─ExchangeReceiver(Build) 7976.02 batchCop[tiflash] ", + " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#31, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#31", + " │ └─Projection 7976.02 batchCop[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5", + " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:sum(Column#15)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", + " │ └─ExchangeReceiver 7976.02 batchCop[tiflash] ", + " │ └─ExchangeSender 7976.02 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─HashAgg 7976.02 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#15", + " │ └─Selection 9970.03 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 7984.01 batchCop[tiflash] ", + " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#32, collate: binary], [name: Column#33, collate: binary]", + " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3, cast(test.t.c3, decimal(10,5))->Column#32, cast(test.t.c1, decimal(40,20))->Column#33", + " └─Projection 7984.01 batchCop[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3", + " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:sum(Column#23)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", + " └─ExchangeReceiver 7984.01 batchCop[tiflash] ", + " └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c1, collate: binary]", + " └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#23", + " └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c2 and t1.c3 = t2.c3 and t1.c4 = t2.c4 and t1.c5 = t2.c5", + "Plan": [ + "TableReader 12462.54 root data:ExchangeSender", + "└─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c2) eq(test.t.c3, test.t.c3) eq(test.t.c4, test.t.c4) eq(test.t.c5, test.t.c5)]", + " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", + " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", + " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c4, collate: binary], [name: test.t.c5, collate: binary]", + " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5", + "Plan": [ + "Projection 12462.54 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 12462.54 root data:ExchangeSender", + " └─ExchangeSender 12462.54 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12462.54 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1) eq(test.t.c3, test.t.c2) eq(test.t.c1, test.t.c3) eq(test.t.c3, test.t.c4) eq(test.t.c5, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", + " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#13, cast(test.t.c1, decimal(10,5))->Column#15", + " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", + " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", + " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#14, cast(test.t.c3, decimal(10,5))->Column#16, cast(test.t.c1, decimal(40,20))->Column#17", + " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t t1 join t t2 on t1.c1 + t1.c2 = t2.c2 / t2.c3", + "Plan": [ + "Projection 12500.00 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 12500.00 root data:ExchangeSender", + " └─ExchangeSender 12500.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12500.00 cop[tiflash] inner join, equal:[eq(Column#13, Column#14)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, Column#13, cast(Column#13, decimal(17,9) BINARY)->Column#17", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, plus(test.t.c1, test.t.c2)->Column#13", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#14, collate: binary]", + " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, div(test.t.c2, test.t.c3)->Column#14", + " └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t t1 where exists (select * from t t2 where t1.c1 = t2.c2 and t1.c2 = t2.c3 and t1.c3 = t2.c1 and t1.c4 = t2.c3 and t1.c1 = t2.c5)", + "Plan": [ + "Projection 7984.01 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 7984.01 root data:ExchangeSender", + " └─ExchangeSender 7984.01 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7984.01 cop[tiflash] semi join, equal:[eq(test.t.c1, test.t.c2) eq(test.t.c2, test.t.c3) eq(test.t.c3, test.t.c1) eq(test.t.c4, test.t.c3) eq(test.t.c1, test.t.c5)]", + " ├─ExchangeReceiver(Build) 9970.03 cop[tiflash] ", + " │ └─ExchangeSender 9970.03 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#14, collate: binary], [name: Column#16, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─Projection 9970.03 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c5, cast(test.t.c3, decimal(10,5))->Column#14, cast(test.t.c1, decimal(10,5))->Column#16", + " │ └─Selection 9970.03 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9980.01 cop[tiflash] ", + " └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#13, collate: binary], [name: Column#15, collate: binary], [name: test.t.c4, collate: binary], [name: Column#17, collate: binary]", + " └─Projection 9980.01 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#13, cast(test.t.c3, decimal(10,5))->Column#15, cast(test.t.c1, decimal(40,20))->Column#17", + " └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t t1 left join t t2 on t1.c1 = t2.c2 join t t3 on t2.c5 = t3.c3 right join t t4 on t3.c3 = t4.c4 ", + "Plan": [ + "Projection 19492.21 root test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + "└─TableReader 19492.21 root data:ExchangeSender", + " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#27, collate: binary]", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo", + " └─Projection(Probe) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#25, collate: binary]", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", + " └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c5, collate: binary]", + " └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", + " │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary]", + " │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary]", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' SELECT STRAIGHT_JOIN t1 . col_varchar_64 , t1 . col_char_64_not_null FROM tt AS t1 INNER JOIN( tt AS t2 JOIN tt AS t3 ON(t3 . col_decimal_30_10_key = t2 . col_tinyint)) ON(t3 . col_varchar_64 = t2 . col_varchar_key) WHERE t3 . col_varchar_64 = t1 . col_char_64_not_null GROUP BY 1 , 2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin], [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", + " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_char_64_not_null, collate: utf8mb4_bin]", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_key, collate: utf8mb4_bin]", + " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.tt.col_varchar_64, collate: utf8mb4_bin]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownAggForMPP", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#8)->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] plus(test.t.id, 1)->Column#9", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ count(*) from (select id+1 from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#7)->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg()*/ sum(b) from (select id + 1 as b from t)A", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#8)->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#9)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] cast(plus(test.t.id, 1), decimal(20,0) BINARY)->Column#9", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from t", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#6)->Column#4", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#6", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*), id from t group by id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#7)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*), id + 1 from t group by id + 1", + "Plan": [ + "Projection 8000.00 root Column#4, plus(test.t.id, 1)->Column#5", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:sum(Column#11)->Column#4, funcs:firstrow(Column#12)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:count(1)->Column#11, funcs:firstrow(Column#16)->Column#12", + " └─Projection 10000.00 batchCop[tiflash] test.t.id, plus(test.t.id, 1)->Column#17", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", + "Plan": [ + "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", + "├─HashAgg(Build) 1.00 root funcs:count(Column#11)->Column#7", + "│ └─TableReader 1.00 root data:ExchangeSender", + "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#11", + "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select avg(value) as b,id from t group by id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#9, 0), 1, Column#9), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#10)->Column#9, funcs:sum(Column#11)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#10, funcs:sum(test.t.value)->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(b) from (select avg(value) as b, id from t group by id)A", + "Plan": [ + "HashAgg 1.00 root funcs:sum(Column#20)->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:sum(Column#4)->Column#20", + " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#4", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#18)->Column#17, funcs:sum(Column#19)->Column#4", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#18, funcs:sum(test.t.value)->Column#19", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select id from t group by id having avg(value)>0", + "Plan": [ + "Projection 6400.00 root test.t.id", + "└─Selection 6400.00 root gt(Column#4, 0)", + " └─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#10, funcs:sum(Column#12)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#11, funcs:sum(test.t.value)->Column#12", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select avg(value),id from t group by id having avg(value)>0", + "Plan": [ + "Selection 6400.00 root gt(Column#4, 0)", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select avg(value) +1,id from t group by id", + "Plan": [ + "Projection 8000.00 root plus(Column#4, 1)->Column#5, test.t.id", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] div(Column#4, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#12)->Column#11, funcs:sum(Column#13)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#12, funcs:sum(test.t.value)->Column#13", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 7992.00 batchCop[tiflash] Column#7", + " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#7", + " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", + " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", + " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id, Column#13", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#17", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(distinct value),id from t group by id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#4, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(distinct value),sum(distinct value),id from t group by id", + "Plan": [ + "HashAgg 8000.00 root group by:test.t.id, funcs:count(distinct test.t.value)->Column#4, funcs:sum(distinct test.t.value)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + "└─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select count(distinct value), id from t group by id) as A on A.id = t.id", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(distinct test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select count(1/value), id from t group by id) as A on A.id = t.id", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─Projection(Build) 7992.00 batchCop[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", + " │ └─Projection 9990.00 batchCop[tiflash] div(1, test.t.value)->Column#18, test.t.id", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(id) from (select value, id from t where id > value group by id, value)A group by value /*the exchange should have only one partition column: test.t.value*/", + "Plan": [ + "TableReader 6400.00 root data:ExchangeSender", + "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 batchCop[tiflash] Column#4", + " └─HashAgg 6400.00 batchCop[tiflash] group by:Column#22, funcs:sum(Column#21)->Column#4", + " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#21, test.t.value", + " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", + " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", + " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", + " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", + " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select /*+hash_agg()*/ sum(B.value) from t as B where B.id+1 > (select count(*) from t where t.id= B.id and t.value=B.value) group by B.id /*the exchange should have only one partition column: test.t.id*/", + "Plan": [ + "TableReader 6400.00 root data:ExchangeSender", + "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 6400.00 batchCop[tiflash] Column#8", + " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#8", + " └─Selection 8000.00 batchCop[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))", + " └─HashJoin 10000.00 batchCop[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", + " ├─Projection(Build) 7984.01 batchCop[tiflash] Column#7, test.t.id, test.t.value", + " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7984.01 batchCop[tiflash] ", + " │ └─ExchangeSender 7984.01 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7984.01 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24", + " │ └─Selection 9980.01 batchCop[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:B keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(distinct value) from t", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#4", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#4)->Column#5", + " └─Projection 1.00 batchCop[tiflash] Column#4", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(distinct value), count(value), avg(value) from t", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#4, Column#5, div(Column#6, cast(case(eq(Column#7, 0), 1, Column#7), decimal(20,0) BINARY))->Column#6", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4, funcs:sum(Column#8)->Column#5, funcs:sum(Column#9)->Column#7, funcs:sum(Column#10)->Column#6", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, funcs:count(test.t.value)->Column#8, funcs:count(test.t.value)->Column#9, funcs:sum(test.t.value)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestMppAggTopNWithJoin", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from t join ( select count(*), id from t group by id) as A on A.id = t.id", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select count(*)+id as v from t group by id) as A on A.v = t.id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, Column#8)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 batchCop[tiflash] plus(Column#7, test.t.id)->Column#8", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(Column#7, test.t.id)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#7, test.t.id", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#11", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select count(*) as v, id from t group by value,id having value+v <10) as A on A.id = t.id", + "Plan": [ + "Projection 7992.00 root test.t.id, test.t.value, Column#7, test.t.id", + "└─TableReader 7992.00 root data:ExchangeSender", + " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 6393.60 batchCop[tiflash] ", + " │ └─ExchangeSender 6393.60 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 6393.60 batchCop[tiflash] lt(plus(test.t.value, cast(Column#7, decimal(20,0) BINARY)), 10)", + " │ └─Projection 7992.00 batchCop[tiflash] Column#7, test.t.id, test.t.value", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#10)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary], [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#10", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join ( select /*+ hash_agg()*/ count(*) as a from t) as A on A.a = t.id", + "Plan": [ + "HashJoin 1.25 root inner join, equal:[eq(test.t.id, Column#7)]", + "├─HashAgg(Build) 1.00 root funcs:count(Column#10)->Column#7", + "│ └─TableReader 1.00 root data:ExchangeSender", + "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#10", + "│ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select sum(b) from (select t.id, t1.id as b from t join t t1 on t.id=t1.id)A group by id", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 7992.00 batchCop[tiflash] Column#7", + " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7", + " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#11, funcs:sum(Column#10)->Column#8", + " └─Projection 12487.50 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#10, test.t.id", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(value),id from t group by id)B on C.id=B.id", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#7, test.t.id", + " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", + " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#17)->Column#11, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#33, funcs:sum(Column#32)->Column#17", + " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 batchCop[tiflash] test.t.id, Column#13", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#16)->Column#13", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:count(1)->Column#16", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", + "Plan": [ + "TopN 1.00 root test.t.value, offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 1.00 batchCop[tiflash] test.t.value, offset:0, count:1", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", + "Plan": [ + "Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value", + "└─TopN 1.00 root Column#8, offset:0, count:1", + " └─Projection 1.00 root test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#8", + " └─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value", + " └─TopN 1.00 batchCop[tiflash] Column#7, offset:0, count:1", + " └─Projection 12487.50 batchCop[tiflash] test.t.id, test.t.value, test.t.id, test.t.value, mod(test.t.value, 100)->Column#7", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", + "Plan": [ + "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "└─TopN 20.00 root test.t.value, offset:0, count:20", + " └─TableReader 20.00 root data:ExchangeSender", + " └─ExchangeSender 20.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TopN 20.00 batchCop[tiflash] test.t.value, offset:0, count:20", + " └─HashJoin 12487.50 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "Plan": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 cop[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "Plan": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 cop[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1", + "Plan": [ + "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "└─Limit 20.00 root offset:0, count:20", + " └─TableReader 20.00 root data:ExchangeSender", + " └─ExchangeSender 20.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 20.00 cop[tiflash] offset:0, count:20", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 16.02 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIndexMergeSerial", + "Cases": [ + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and b+2>1)", + "Plan": [ + "IndexMerge 8.00 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where a =1 or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(a)=1) or (b=1 and length(b)=1)", + "Plan": [ + "IndexMerge 8.00 root ", + "├─Selection(Build) 0.80 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─Selection(Build) 0.80 cop[tikv] 1", + "│ └─IndexRangeScan 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─TableRowIDScan(Probe) 8.00 cop[tikv] table:t keep order:false" + ], + "Warnings": null + }, + { + "SQL": "desc format='brief' select /*+ use_index_merge(t) */ * from t where (a=1 and length(b)=1) or (b=1 and length(a)=1)", + "Plan": [ + "IndexMerge 0.29 root ", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:a(a) range:[1,1], keep order:false", + "├─IndexRangeScan(Build) 1.00 cop[tikv] table:t, index:b(b) range:[1,1], keep order:false", + "└─Selection(Probe) 0.29 cop[tikv] or(and(eq(test.t.a, 1), eq(length(cast(test.t.b, var_string(20))), 1)), and(eq(test.t.b, 1), eq(length(cast(test.t.a, var_string(20))), 1)))", + " └─TableRowIDScan 1.90 cop[tikv] table:t keep order:false" + ], + "Warnings": null + } + ] + }, + { + "Name": "TestLimitIndexLookUpKeepOrder", + "Cases": [ + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b,c limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t where a = 1 and b > 2 and b < 10 and d = 10 order by b desc, c desc limit 10", + "Plan": [ + "Limit 0.00 root offset:0, count:10", + "└─Projection 0.00 root test.t.a, test.t.b, test.t.c, test.t.d", + " └─IndexLookUp 0.00 root ", + " ├─IndexRangeScan(Build) 2.50 cop[tikv] table:t, index:idx(a, b, c) range:(1 2,1 10), keep order:true, desc, stats:pseudo", + " └─Selection(Probe) 0.00 cop[tikv] eq(test.t.d, 10)", + " └─TableRowIDScan 2.50 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestIssue23887", + "Cases": [ + { + "SQL": "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t", + "Plan": [ + "HashJoin 10000.00 root CARTESIAN left outer semi join, other cond:eq(2, test.t.b)", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "└─Projection(Probe) 10000.00 root 1->Column#27", + " └─Apply 10000.00 root CARTESIAN left outer join", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 root 1->Column#25", + " └─Limit 1.00 root offset:0, count:1", + " └─TableReader 1.00 root data:Limit", + " └─Limit 1.00 cop[tikv] offset:0, count:1", + " └─TableFullScan 1.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ], + "Res": [ + "1", + "1" + ] + } + ] + }, + { + "Name": "TestMergeContinuousSelections", + "Cases": [ + { + "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", + "Plan": [ + "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", + "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", + " ├─Selection(Build) 0.80 root ne(Column#27, 0)", + " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", + " │ └─TableReader 1.00 root data:ExchangeSender", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(20,0) BINARY)->Column#43", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 12487.50 root data:ExchangeSender", + " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" + ] + } + ] + }, + { + "Name": "TestPushDownGroupConcatToTiFlash", + "Cases": [ + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#7 separator \",\")->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#9, Column#10, Column#11 separator \",\")->Column#7", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#11", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_1) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#8, Column#9, Column#10 order by Column#11 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#12)->Column#7", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#10, test.ts.col_0, test.ts.col_1", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_0) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#14, Column#15, Column#16 order by Column#17 separator \",\")->Column#5, funcs:sum(Column#18)->Column#6, funcs:max(Column#19)->Column#7", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#16, test.ts.col_0, Column#12, Column#13", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.id, funcs:count(1)->Column#12, funcs:max(test.ts.col_0)->Column#13", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#12, test.ts.col_2", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7, Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_2", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#10, funcs:group_concat(distinct Column#6, Column#7, Column#8 order by Column#9 separator \",\")->Column#5", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#8, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_0, test.ts.col_1, test.ts.col_2, test.ts.id, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_1, id order by col_0) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#9, funcs:group_concat(distinct Column#6, Column#7 order by Column#8 separator \",\")->Column#5", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#7, test.ts.col_0, test.ts.col_2", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, test.ts.col_2, test.ts.id, funcs:firstrow(test.ts.col_0)->test.ts.col_0", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(*),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#11, 0), 1, Column#11), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(Column#13, Column#14, Column#15 order by Column#16 separator \",\")->Column#5, funcs:count(1)->Column#6, funcs:min(Column#17)->Column#7, funcs:count(Column#18)->Column#11, funcs:sum(Column#19)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#15, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#19, test.ts.col_2", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(*),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#24, 0), 1, Column#24), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#44, funcs:group_concat(distinct Column#36, Column#37, Column#38 order by Column#39 separator \",\")->Column#5, funcs:sum(Column#40)->Column#6, funcs:max(Column#41)->Column#7, funcs:sum(Column#42)->Column#24, funcs:sum(Column#43)->Column#8", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#38, test.ts.col_0, Column#25, Column#26, Column#27, Column#28, test.ts.col_2", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#32, Column#33, Column#34, Column#35, funcs:count(1)->Column#25, funcs:max(Column#29)->Column#26, funcs:count(Column#30)->Column#27, funcs:sum(Column#31)->Column#28", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#31, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id order by col_0),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#10, 0), 1, Column#10), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#11, Column#12, Column#13 order by Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:count(Column#17)->Column#10, funcs:sum(Column#18)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#13, test.ts.col_0, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#18", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id order by col_0),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#30, Column#31, Column#32 order by Column#33 separator \",\")->Column#5, funcs:sum(Column#34)->Column#6, funcs:max(Column#35)->Column#7, funcs:sum(Column#36)->Column#18, funcs:sum(Column#37)->Column#8", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#32, test.ts.col_0, Column#19, Column#20, Column#21, Column#22", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#27, 0), 1, Column#27), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_2, funcs:group_concat(Column#28 separator \",\")->Column#5, funcs:sum(Column#29)->Column#6, funcs:min(Column#30)->Column#7, funcs:sum(Column#31)->Column#27, funcs:sum(Column#32)->Column#8", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#40, funcs:group_concat(Column#33, Column#34, Column#35 separator \",\")->Column#28, funcs:count(Column#36)->Column#29, funcs:min(Column#37)->Column#30, funcs:count(Column#38)->Column#31, funcs:sum(Column#39)->Column#32", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#39, test.ts.col_2", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#24, 0), 1, Column#24), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#44, funcs:group_concat(distinct Column#37, Column#38, Column#39 separator \",\")->Column#5, funcs:sum(Column#40)->Column#6, funcs:max(Column#41)->Column#7, funcs:sum(Column#42)->Column#24, funcs:sum(Column#43)->Column#8", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#39, Column#25, Column#26, Column#27, Column#28, test.ts.col_2", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#33, Column#34, Column#35, Column#36, funcs:count(Column#29)->Column#25, funcs:max(Column#30)->Column#26, funcs:count(Column#31)->Column#27, funcs:sum(Column#32)->Column#28", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#32, test.ts.col_2, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),min(col_0),avg(id) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(Column#14 separator \",\")->Column#5, funcs:count(Column#15)->Column#6, funcs:min(Column#16)->Column#7, funcs:avg(Column#17, Column#18)->Column#8", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#24, Column#25, Column#26 separator \",\")->Column#14, funcs:count(Column#27)->Column#15, funcs:min(Column#28)->Column#16, funcs:count(Column#29)->Column#17, funcs:sum(Column#30)->Column#18", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#26, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#30", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#30, Column#31, Column#32 separator \",\")->Column#5, funcs:sum(Column#33)->Column#6, funcs:max(Column#34)->Column#7, funcs:sum(Column#35)->Column#18, funcs:sum(Column#36)->Column#8", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#32, Column#19, Column#20, Column#21, Column#22", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#27, Column#28, Column#29, funcs:count(Column#23)->Column#19, funcs:max(Column#24)->Column#20, funcs:count(Column#25)->Column#21, funcs:sum(Column#26)->Column#22", + " └─Projection 10000.00 batchCop[tiflash] test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#26, test.ts.col_0, test.ts.col_1, test.ts.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_1, id),count(distinct id),group_concat(col_0 order by 1),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:group_concat(Column#21, Column#22, Column#23 separator \",\")->Column#5, funcs:count(Column#24)->Column#6, funcs:group_concat(Column#25 order by Column#26 separator \",\")->Column#7, funcs:count(Column#27)->Column#17, funcs:sum(Column#28)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#23, test.ts.id, test.ts.col_0, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_2", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0),count(distinct id),group_concat(col_1, id order by 1,2),avg(id) from ts group by col_2", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, div(Column#8, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#8", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#25, funcs:group_concat(distinct Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:count(Column#23)->Column#14, funcs:sum(Column#24)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#24, test.ts.col_2", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_2, collate: utf8mb4_bin]", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, id),count(distinct id),group_concat(col_1, id order by 1,2),min(col_0),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(Column#18, Column#19 separator \",\")->Column#5, funcs:count(Column#20)->Column#6, funcs:group_concat(Column#21, Column#22 order by Column#23, Column#24 separator \",\")->Column#7, funcs:min(Column#25)->Column#8, funcs:count(Column#26)->Column#15, funcs:sum(Column#27)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, cast(test.ts.id, var_string(20))->Column#19, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#22, test.ts.col_1, test.ts.id, test.ts.col_0, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#27", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct id),group_concat(col_1, id order by 1,2),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#13, 0), 1, Column#13), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#15, Column#16, Column#17 separator \",\")->Column#5, funcs:count(Column#18)->Column#6, funcs:group_concat(Column#19, Column#20 order by Column#21, Column#22 separator \",\")->Column#7, funcs:max(Column#23)->Column#8, funcs:count(Column#24)->Column#13, funcs:sum(Column#25)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#17, test.ts.id, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#20, test.ts.col_1, test.ts.id, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#25", + " └─ExchangeReceiver 10000.00 batchCop[tiflash] ", + " └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#33, Column#34, Column#35 separator \",\")->Column#5, funcs:count(distinct Column#36)->Column#6, funcs:group_concat(Column#37 separator \",\")->Column#7, funcs:max(Column#38)->Column#8, funcs:sum(Column#39)->Column#19, funcs:sum(Column#40)->Column#9", + " └─Projection 1.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#35, test.ts.col_2, Column#20, Column#21, Column#22, Column#23", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#29, Column#30, Column#31, Column#32, funcs:group_concat(Column#24, Column#25 separator \",\")->Column#20, funcs:max(Column#26)->Column#21, funcs:count(Column#27)->Column#22, funcs:sum(Column#28)->Column#23", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#25, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#28, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0, col_1, id),count(distinct col_2),group_concat(col_1, id),max(col_1),avg(id) from ts group by col_0", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, Column#6, Column#7, Column#8, div(Column#9, cast(case(eq(Column#25, 0), 1, Column#25), decimal(20,0) BINARY))->Column#9", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#47, funcs:group_concat(distinct Column#39, Column#40, Column#41 separator \",\")->Column#5, funcs:count(distinct Column#42)->Column#6, funcs:group_concat(Column#43 separator \",\")->Column#7, funcs:max(Column#44)->Column#8, funcs:sum(Column#45)->Column#25, funcs:sum(Column#46)->Column#9", + " └─Projection 8000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_1, cast(test.ts.id, var_string(20))->Column#41, test.ts.col_2, Column#26, Column#27, Column#28, Column#29, test.ts.col_0", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_0, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#35, Column#36, Column#37, Column#38, funcs:group_concat(Column#30, Column#31 separator \",\")->Column#26, funcs:max(Column#32)->Column#27, funcs:count(Column#33)->Column#28, funcs:sum(Column#34)->Column#29", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_1, cast(test.ts.id, var_string(20))->Column#31, test.ts.col_1, test.ts.id, cast(test.ts.id, decimal(14,4) BINARY)->Column#34, test.ts.col_0, test.ts.col_1, test.ts.id, test.ts.col_2", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, Column#11", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'01') from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, Column#11", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"01\", 0, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,1) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#10, var_string(20))->Column#12, cast(Column#11, var_string(20))->Column#13", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#9, Column#10 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#8, var_string(20))->Column#9, cast(Column#8, var_string(20))->Column#10", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:0, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,10) from ts group by '010'", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(distinct Column#18, Column#19 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#16, var_string(20))->Column#18, cast(Column#17, var_string(20))->Column#19, Column#15", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: binary]", + " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, 10, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,0) from ts group by '011'", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#16, funcs:group_concat(distinct Column#14, Column#15 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#13, var_string(20))->Column#14, cast(Column#13, var_string(20))->Column#15, Column#12", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", + " └─HashAgg 1.00 batchCop[tiflash] group by:0, 1, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 0,'GG') from ts group by 'GG'", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#20, funcs:group_concat(distinct Column#18, Column#19 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] cast(Column#16, var_string(20))->Column#18, Column#17, Column#15", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#15, collate: binary]", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", 0, 1, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'", + "[types:1292]Truncated incorrect DOUBLE value: 'GG'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG','GG') from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#8, Column#8 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'Gg','GG') from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"Gg\", ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct 'GG-10','GG') from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#10, Column#11 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"GG\", \"GG-10\", ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct '1200-01-01 00:00:00.023',1200) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct Column#12, Column#13 separator \",\")->Column#5", + " └─Projection 1.00 batchCop[tiflash] Column#10, cast(Column#11, var_string(20))->Column#13", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:\"1200-01-01 00:00:00.023\", 1200, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'", + "[types:1292]Truncated incorrect DOUBLE value: '1200-01-01 00:00:00.023'" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0) from ts group by id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(test.ts.col_0, test.ts.col_0 separator \",\")->Column#9", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(col_0, col_0,id) from ts group by id", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.id, funcs:group_concat(Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:group_concat(Column#10, Column#11, Column#12 separator \",\")->Column#9", + " └─Projection 10000.00 batchCop[tiflash] test.ts.col_0, test.ts.col_0, cast(test.ts.id, var_string(20))->Column#12, test.ts.id", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:group_concat(distinct test.ts.col_0 order by Column#8 separator \",\")->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#10, funcs:firstrow(Column#9)->Column#8", + " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#9, test.ts.col_0", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.ts.col_1, funcs:group_concat(distinct test.ts.col_0 order by Column#9 separator \",\")->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#11, Column#12, funcs:firstrow(Column#10)->Column#9", + " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#10, test.ts.col_1, test.ts.col_0", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0>10 order by id<10) from ts group by col_1", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#19, funcs:group_concat(distinct Column#17 order by Column#18 separator \",\")->Column#5", + " └─Projection 8000.00 batchCop[tiflash] cast(Column#12, var_string(20))->Column#17, Column#13, test.ts.col_1", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.ts.col_1, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#15, Column#16, funcs:firstrow(Column#14)->Column#13", + " └─Projection 10000.00 batchCop[tiflash] lt(test.ts.id, 10)->Column#14, test.ts.col_1, gt(cast(test.ts.col_0, double BINARY), 10)->Column#16", + " └─TableFullScan 10000.00 batchCop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable" + ] + }, + { + "SQL": "desc format = 'brief' select /*+ hash_agg(),agg_to_cop() */ group_concat(distinct col_0 order by col_0<=>null) from ts", + "Plan": [ + "HashAgg 1.00 root funcs:group_concat(distinct Column#6 order by Column#7 separator \",\")->Column#5", + "└─Projection 10000.00 root test.ts.col_0, nulleq(test.ts.col_0, )->Column#7", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tiflash] table:ts keep order:false, stats:pseudo" + ], + "Warning": [ + "[planner:1815]Optimizer Hint AGG_TO_COP is inapplicable", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause", + "Scalar function 'nulleq'(signature: NullEQString, return type: bigint(1)) is not supported to push down to tiflash now.", + "Aggregation can not be pushed to tiflash because arguments of AggFunc `group_concat` contains unsupported exprs in order-by clause" + ] + } + ] + }, + { + "Name": "TestRejectSortForMPP", + "Cases": [ + { + "SQL": "desc format = 'brief' select count(*) from (select * from t order by id)a group by name,id order by id", + "Plan": [ + "Projection 8000.00 root Column#5", + "└─Sort 8000.00 root test.t.id", + " └─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5, test.t.id", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select * from t order by id)a group by name order by 1", + "Plan": [ + "Sort 8000.00 root Column#5", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select id,name from t group by id,name order by id,name)a group by name order by 1", + "Plan": [ + "Sort 8000.00 root Column#5", + "└─TableReader 8000.00 root data:ExchangeSender", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.name, funcs:count(1)->Column#5", + " └─Projection 8000.00 batchCop[tiflash] test.t.id, test.t.name", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.name)->test.t.name", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.t.id, test.t.name, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select id from t group by id order by id)a join t on a.id=t.id order by 1", + "Plan": [ + "Sort 9990.00 root test.t.id", + "└─TableReader 9990.00 root data:ExchangeSender", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 batchCop[tiflash] test.t.id, test.t.id, test.t.value, test.t.name", + " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select * from t order by id)a join t on a.id=t.id order by 1", + "Plan": [ + "Sort 12487.50 root test.t.id", + "└─TableReader 12487.50 root data:ExchangeSender", + " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 cop[tiflash] test.t.id, test.t.value, test.t.name, test.t.id, test.t.value, test.t.name", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from ((select id from t order by 1) union all (select id+1 from t order by 1))c", + "Plan": [ + "TableReader 20000.00 root data:ExchangeSender", + "└─ExchangeSender 20000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(test.t.id, bigint(20) BINARY)->Column#10", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] plus(test.t.id, 1)->Column#10", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from ((select count(*) from (select id,name from t order by id)a group by name,id order by id) union all (select id+1 from t order by 1))c", + "Plan": [ + "TableReader 18000.00 root data:ExchangeSender", + "└─ExchangeSender 18000.00 cop[tiflash] ExchangeType: PassThrough", + " └─Union 18000.00 cop[tiflash] ", + " ├─Projection 8000.00 cop[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", + " │ └─Projection 8000.00 cop[tiflash] Column#5", + " │ └─Projection 8000.00 cop[tiflash] Column#5, test.t.id", + " │ └─HashAgg 8000.00 cop[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 cop[tiflash] ", + " │ └─ExchangeSender 8000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 cop[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", + " └─Projection 10000.00 cop[tiflash] plus(test.t.id, 1)->Column#11", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from (select * from t order by id)a order by name", + "Plan": [ + "Sort 10000.00 root test.t.name", + "└─TableReader 10000.00 root data:ExchangeSender", + " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: PassThrough", + " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 92f425c428531..df15027594cd4 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -886,7 +886,7 @@ }, { "SQL": "select sum(to_base64(e)) from t where c = 1", - "Best": "IndexReader(Index(t.c_d_e)[[1,1]])->Projection->StreamAgg" + "Best": "IndexReader(Index(t.c_d_e)[[1,1]]->StreamAgg)->StreamAgg" }, { "SQL": "select (select count(1) k from t s where s.a = t.a having k != 0) from t", @@ -894,7 +894,7 @@ }, { "SQL": "select sum(to_base64(e)) from t group by e,d,c order by c", - "Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]])->Projection->StreamAgg->Projection" + "Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]]->StreamAgg)->StreamAgg->Projection" }, { "SQL": "select sum(e+1) from t group by e,d,c order by c", @@ -902,7 +902,7 @@ }, { "SQL": "select sum(to_base64(e)) from t group by e,d,c order by c,e", - "Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]])->Projection->StreamAgg->Sort->Projection" + "Best": "IndexReader(Index(t.c_d_e)[[NULL,+inf]]->StreamAgg)->StreamAgg->Sort->Projection" }, { "SQL": "select sum(e+1) from t group by e,d,c order by c,e", @@ -1075,11 +1075,11 @@ }, { "SQL": "select a from t where c_str not like 'abc'", - "Best": "IndexReader(Index(t.c_d_e_str)[[-inf,\"abc\") (\"abc\",+inf]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[NULL,+inf]]->Sel([not(like(test.t.c_str, abc, 92))]))->Projection" }, { "SQL": "select a from t where not (c_str like 'abc' or c_str like 'abd')", - "Best": "IndexReader(Index(t.c_d_e_str)[[-inf,\"abc\") (\"abc\",\"abd\") (\"abd\",+inf]])->Projection" + "Best": "IndexReader(Index(t.c_d_e_str)[[NULL,+inf]]->Sel([and(not(like(test.t.c_str, abc, 92)), not(like(test.t.c_str, abd, 92)))]))->Projection" }, { "SQL": "select a from t where c_str like '_abc'", @@ -2689,7 +2689,7 @@ " └─Projection 8000.00 batchCop[tiflash] Column#5", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#5", " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#14, funcs:count(1)->Column#13", " └─Projection 10000.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#14", " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" @@ -2746,7 +2746,7 @@ " └─Projection 1.00 batchCop[tiflash] Column#7", " └─HashAgg 1.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#7", " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", " └─HashAgg 1.00 batchCop[tiflash] group by:Column#15, funcs:count(Column#14)->Column#13", " └─Projection 1.00 batchCop[tiflash] Column#5, plus(Column#6, 1)->Column#15", " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6", @@ -2876,7 +2876,7 @@ " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", @@ -2897,7 +2897,7 @@ " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", @@ -2918,7 +2918,7 @@ " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", @@ -2927,7 +2927,7 @@ " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: N/A]", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" @@ -2988,11 +2988,11 @@ "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: N/A]", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", " │ └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: N/A]", + " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", " └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" ] @@ -3005,18 +3005,18 @@ " └─Projection 8000.00 batchCop[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: N/A]", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: N/A]", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" @@ -3030,18 +3030,18 @@ " └─Projection 8000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#10, Column#9", " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: N/A]", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: binary]", " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: N/A]", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: binary]", " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" @@ -3054,24 +3054,24 @@ "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashJoin 6400.00 batchCop[tiflash] inner join, equal:[eq(Column#6, Column#12)]", " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", - " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: N/A]", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: binary]", " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: N/A]", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: binary]", " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 6400.00 batchCop[tiflash] ", - " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: binary]", " └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", " └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: N/A]", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary]", " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json index 06c90571e11fe..98c7b9b9b5985 100644 --- a/planner/core/testdata/plan_suite_unexported_in.json +++ b/planner/core/testdata/plan_suite_unexported_in.json @@ -131,7 +131,8 @@ "select t1.b from t t1 where t1.b in (select t2.b from t t2 where t2.a = t1.a order by t2.a)", "select t1.b from t t1 where exists(select t2.b from t t2 where t2.a = t1.a order by t2.a)", // `Sort` will not be eliminated, if it is not the top level operator. - "select t1.b from t t1 where t1.b = (select t2.b from t t2 where t2.a = t1.a order by t2.a limit 1)" + "select t1.b from t t1 where t1.b = (select t2.b from t t2 where t2.a = t1.a order by t2.a limit 1)", + "select (select 1 from t t1 where t1.a = t2.a) from t t2" ] }, { diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index 4fc39a9e7d4d0..44b29b2594347 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -117,7 +117,8 @@ "Join{DataScan(t1)->DataScan(t2)->Aggr(max(test.t.a),firstrow(test.t.b))}(test.t.b,test.t.b)->Projection->Sel([eq(test.t.b, Column#25)])->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)(test.t.b,test.t.b)->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", - "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t.a, test.t.a)])->Projection->Sort->Limit}->Projection->Sel([eq(test.t.b, test.t.b)])->Projection" + "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t.a, test.t.a)])->Projection->Sort->Limit}->Projection->Sel([eq(test.t.b, test.t.b)])->Projection", + "Apply{DataScan(t2)->DataScan(t1)->Sel([eq(test.t.a, test.t.a)])->Projection}->Projection" ] }, { diff --git a/planner/core/testdata/stats_suite_out.json b/planner/core/testdata/stats_suite_out.json index 369fbe4fb0adf..26f0f2fe5a193 100644 --- a/planner/core/testdata/stats_suite_out.json +++ b/planner/core/testdata/stats_suite_out.json @@ -24,7 +24,7 @@ }, { "SQL": "select count(1) from t1 where cos(a) > 0 group by a, b", - "AggInput": "[]", + "AggInput": "[{[1 2] 3.2}]", "JoinInput": "" }, { diff --git a/planner/optimize.go b/planner/optimize.go index 299f93e4b29ab..83824af121199 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -99,6 +99,16 @@ func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindRecord func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (plannercore.Plan, types.NameSlice, error) { sessVars := sctx.GetSessionVars() + if !sctx.GetSessionVars().InRestrictedSQL && variable.RestrictedReadOnly.Load() || variable.VarTiDBSuperReadOnly.Load() { + allowed, err := allowInReadOnlyMode(sctx, node) + if err != nil { + return nil, nil, err + } + if !allowed { + return nil, nil, errors.Trace(core.ErrSQLInReadOnlyMode) + } + } + // Because for write stmt, TiFlash has a different results when lock the data in point get plan. We ban the TiFlash // engine in not read only stmt. if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(node, sessVars) { @@ -351,16 +361,6 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in return nil, nil, 0, err } - if !sctx.GetSessionVars().InRestrictedSQL && variable.RestrictedReadOnly.Load() || variable.VarTiDBSuperReadOnly.Load() { - allowed, err := allowInReadOnlyMode(sctx, node) - if err != nil { - return nil, nil, 0, err - } - if !allowed { - return nil, nil, 0, errors.Trace(core.ErrSQLInReadOnlyMode) - } - } - // Handle the execute statement. if execPlan, ok := p.(*plannercore.Execute); ok { err := execPlan.OptimizePreparedPlan(ctx, sctx, is) diff --git a/planner/util/path.go b/planner/util/path.go index 9a0b4207d1314..a25d8a0696034 100644 --- a/planner/util/path.go +++ b/planner/util/path.go @@ -64,6 +64,9 @@ type AccessPath struct { Forced bool // IsSingleScan indicates whether the path is a single index/table scan or table access after index scan. IsSingleScan bool + + // Maybe added in model.IndexInfo better, but the cache of model.IndexInfo may lead side effect + IsUkShardIndexPath bool } // IsTablePath returns true if it's IntHandlePath or CommonHandlePath. diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index a5b806aa67ae2..f6d09ffc5c612 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -30,6 +30,7 @@ var connection int32 // Validate implements TiDB plugin's Validate SPI. // It is called before OnInit +// nolint: unused, deadcode func Validate(ctx context.Context, m *plugin.Manifest) error { fmt.Println("## conn_ip_example Validate called ##") fmt.Printf("---- context: %s\n", ctx) @@ -37,6 +38,7 @@ func Validate(ctx context.Context, m *plugin.Manifest) error { } // OnInit implements TiDB plugin's OnInit SPI. +// nolint: unused, deadcode func OnInit(ctx context.Context, manifest *plugin.Manifest) error { fmt.Println("## conn_ip_example OnInit called ##") fmt.Printf("---- context: %s\n", ctx) @@ -78,6 +80,7 @@ func OnInit(ctx context.Context, manifest *plugin.Manifest) error { } // OnShutdown implements TiDB plugin's OnShutdown SPI. +// nolint: unused, deadcode func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { fmt.Println("## conn_ip_example OnShutdown called ##") fmt.Printf("---- context: %s\n", ctx) @@ -87,6 +90,7 @@ func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { } // OnGeneralEvent implements TiDB Audit plugin's OnGeneralEvent SPI. +// nolint: unused, deadcode func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { fmt.Println("## conn_ip_example OnGeneralEvent called ##") if sctx != nil { @@ -112,6 +116,7 @@ func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugi } // OnConnectionEvent implements TiDB Audit plugin's OnConnectionEvent SPI. +// nolint: unused, deadcode func OnConnectionEvent(ctx context.Context, event plugin.ConnectionEvent, info *variable.ConnectionInfo) error { var reason string if r := ctx.Value(plugin.RejectReasonCtxValue{}); r != nil { diff --git a/plugin/main_test.go b/plugin/main_test.go index cd0e57872a41b..c06337b40927c 100644 --- a/plugin/main_test.go +++ b/plugin/main_test.go @@ -25,7 +25,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("time.Sleep"), } diff --git a/plugin/plugin.go b/plugin/plugin.go index 572b676aa3450..b3aac9694193f 100644 --- a/plugin/plugin.go +++ b/plugin/plugin.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -46,6 +46,7 @@ type plugins struct { } // clone deep copies plugins info. +// nolint: unused func (p *plugins) clone() *plugins { np := &plugins{ plugins: make(map[Kind][]Plugin, len(p.plugins)), diff --git a/privilege/privileges/main_test.go b/privilege/privileges/main_test.go index 9819d73c47670..6e31a437983aa 100644 --- a/privilege/privileges/main_test.go +++ b/privilege/privileges/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/server/conn_test.go b/server/conn_test.go index a599ccee48702..7d2571882b7ad 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -27,14 +27,11 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/arena" "github.com/pingcap/tidb/util/chunk" @@ -718,16 +715,6 @@ func TestPrefetchPointKeys(t *testing.T) { tk.MustQuery("select * from prefetch").Check(testkit.Rows("1 1 3", "2 2 6", "3 3 5")) } -func testGetTableByName(t *testing.T, ctx sessionctx.Context, db, table string) table.Table { - dom := domain.GetDomain(ctx) - // Make sure the table schema is the new schema. - err := dom.Reload() - require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) - require.NoError(t, err) - return tbl -} - func TestTiFlashFallback(t *testing.T) { store, clean := testkit.CreateMockStore(t, mockstore.WithClusterInspector(func(c testutils.Cluster) { @@ -757,7 +744,7 @@ func TestTiFlashFallback(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int not null primary key, b int not null)") tk.MustExec("alter table t set tiflash replica 1") - tb := testGetTableByName(t, tk.Session(), "test", "t") + tb := testkit.TestGetTableByName(t, tk.Session(), "test", "t") err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) require.NoError(t, err) diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 99a6de439537f..58469e10dd43a 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -384,7 +384,7 @@ func convertColumnInfo(fld *ast.ResultField) (ci *ColumnInfo) { if fld.Column.Tp == mysql.TypeNewDecimal { // Consider the negative sign. ci.ColumnLength++ - if fld.Column.Decimal > int(types.DefaultFsp) { + if fld.Column.Decimal > types.DefaultFsp { // Consider the decimal point. ci.ColumnLength++ } diff --git a/server/driver_tidb_test.go b/server/driver_tidb_test.go index b7126b31e00b4..a16168cd5017d 100644 --- a/server/driver_tidb_test.go +++ b/server/driver_tidb_test.go @@ -25,14 +25,14 @@ import ( "github.com/stretchr/testify/require" ) -func createColumnByTypeAndLen(tp byte, len uint32) *ColumnInfo { +func createColumnByTypeAndLen(tp byte, cl uint32) *ColumnInfo { return &ColumnInfo{ Schema: "test", Table: "dual", OrgTable: "", Name: "a", OrgName: "a", - ColumnLength: len, + ColumnLength: cl, Charset: uint16(mysql.CharsetNameToID(charset.CharsetUTF8)), Flag: uint16(mysql.UnsignedFlag), Decimal: uint8(0), diff --git a/server/http_handler.go b/server/http_handler.go index e87ff7eec88b7..23df8471533e2 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -739,6 +739,28 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { cfg.PessimisticTxn.DeadlockHistoryCollectRetryable = collectRetryable config.StoreGlobalConfig(cfg) } + if mutationChecker := req.Form.Get("tidb_enable_mutation_checker"); mutationChecker != "" { + s, err := session.CreateSession(h.Store) + if err != nil { + writeError(w, err) + return + } + defer s.Close() + + switch mutationChecker { + case "0": + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableMutationChecker, variable.Off) + case "1": + err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableMutationChecker, variable.On) + default: + writeError(w, errors.New("illegal argument")) + return + } + if err != nil { + writeError(w, err) + return + } + } } else { writeData(w, config.GetGlobalConfig()) } @@ -1782,7 +1804,7 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. respValue := resp.Value var result interface{} = resp if respValue.Info != nil { - datas := make(map[string][]map[string]string) + datas := make(map[string]map[string]string) for _, w := range respValue.Info.Writes { if len(w.ShortValue) > 0 { datas[strconv.FormatUint(w.StartTs, 10)], err = h.decodeMvccData(w.ShortValue, colMap, tb.Meta()) @@ -1811,16 +1833,16 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. return result, nil } -func (h mvccTxnHandler) decodeMvccData(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) ([]map[string]string, error) { +func (h mvccTxnHandler) decodeMvccData(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) (map[string]string, error) { rs, err := tablecodec.DecodeRowToDatumMap(bs, colMap, time.UTC) - var record []map[string]string + record := make(map[string]string, len(tb.Columns)) for _, col := range tb.Columns { if c, ok := rs[col.ID]; ok { data := "nil" if !c.IsNull() { data, err = c.ToString() } - record = append(record, map[string]string{col.Name.O: data}) + record[col.Name.O] = data } } return record, err diff --git a/server/http_status.go b/server/http_status.go index 79f7debd13ac8..0ed6dd07543a8 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -453,17 +453,20 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) { httpL := m.Match(cmux.HTTP1Fast()) grpcL := m.Match(cmux.Any()) - s.statusServer = &http.Server{Addr: s.statusAddr, Handler: CorsHandler{handler: serverMux, cfg: s.cfg}} - s.grpcServer = NewRPCServer(s.cfg, s.dom, s) - service.RegisterChannelzServiceToServer(s.grpcServer) + statusServer := &http.Server{Addr: s.statusAddr, Handler: CorsHandler{handler: serverMux, cfg: s.cfg}} + grpcServer := NewRPCServer(s.cfg, s.dom, s) + service.RegisterChannelzServiceToServer(grpcServer) + + s.statusServer = statusServer + s.grpcServer = grpcServer go util.WithRecovery(func() { - err := s.grpcServer.Serve(grpcL) + err := grpcServer.Serve(grpcL) logutil.BgLogger().Error("grpc server error", zap.Error(err)) }, nil) go util.WithRecovery(func() { - err := s.statusServer.Serve(httpL) + err := statusServer.Serve(httpL) logutil.BgLogger().Error("http server error", zap.Error(err)) }, nil) diff --git a/server/main_test.go b/server/main_test.go index 051aba800d82e..8d6ef39bb7c33 100644 --- a/server/main_test.go +++ b/server/main_test.go @@ -59,7 +59,7 @@ func TestMain(m *testing.M) { goleak.IgnoreTopFunction("github.com/pingcap/tidb/server.NewServer.func1"), goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("github.com/go-sql-driver/mysql.(*mysqlConn).startWatcher.func1"), } diff --git a/server/packetio.go b/server/packetio.go index 8ab9e40042fd0..250ae024cd362 100644 --- a/server/packetio.go +++ b/server/packetio.go @@ -49,8 +49,8 @@ import ( const defaultWriterSize = 16 * 1024 var ( - readPacketBytes = metrics.PacketIOHistogram.WithLabelValues("read") - writePacketBytes = metrics.PacketIOHistogram.WithLabelValues("write") + readPacketBytes = metrics.PacketIOCounter.WithLabelValues("read") + writePacketBytes = metrics.PacketIOCounter.WithLabelValues("write") ) // packetIO is a helper to read and write data in packet format. @@ -120,7 +120,7 @@ func (p *packetIO) readPacket() ([]byte, error) { } if len(data) < mysql.MaxPayloadLen { - readPacketBytes.Observe(float64(len(data))) + readPacketBytes.Add(float64(len(data))) return data, nil } @@ -138,14 +138,14 @@ func (p *packetIO) readPacket() ([]byte, error) { } } - readPacketBytes.Observe(float64(len(data))) + readPacketBytes.Add(float64(len(data))) return data, nil } // writePacket writes data that already have header func (p *packetIO) writePacket(data []byte) error { length := len(data) - 4 - writePacketBytes.Observe(float64(len(data))) + writePacketBytes.Add(float64(len(data))) for length >= mysql.MaxPayloadLen { data[0] = 0xff diff --git a/server/server_test.go b/server/server_test.go index 034587d7be1f7..c09bfbd7a9c20 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -400,10 +400,11 @@ func (cli *testServerClient) runTestLoadDataWithSelectIntoOutfile(t *testing.T, } func (cli *testServerClient) runTestLoadDataForSlowLog(t *testing.T, server *Server) { - path := "/tmp/load_data_test.csv" - fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) + t.Skip("unstable test") + fp, err := os.CreateTemp("", "load_data_test.csv") require.NoError(t, err) require.NotNil(t, fp) + path := fp.Name() defer func() { err = fp.Close() require.NoError(t, err) @@ -460,29 +461,29 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(t *testing.T, server *Ser }) } -func (cli *testServerClient) prepareLoadDataFile(t *testing.T, path string, rows ...string) { - fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) +func (cli *testServerClient) prepareLoadDataFile(t *testing.T, fp *os.File, rows ...string) { + err := fp.Truncate(0) require.NoError(t, err) - require.NotNil(t, fp) - defer func() { - err = fp.Close() - require.NoError(t, err) - }() + _, err = fp.Seek(0, 0) + require.NoError(t, err) + for _, row := range rows { fields := strings.Split(row, " ") _, err = fp.WriteString(strings.Join(fields, "\t")) + require.NoError(t, err) _, err = fp.WriteString("\n") + require.NoError(t, err) } - require.NoError(t, err) + require.NoError(t, fp.Sync()) } func (cli *testServerClient) runTestLoadDataAutoRandom(t *testing.T) { - path := "/tmp/load_data_txn_error.csv" - - fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) + fp, err := os.CreateTemp("", "load_data_txn_error.csv") require.NoError(t, err) require.NotNil(t, fp) + path := fp.Name() + defer func() { _ = os.Remove(path) }() @@ -534,11 +535,10 @@ func (cli *testServerClient) runTestLoadDataAutoRandom(t *testing.T) { } func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(t *testing.T) { - path := "/tmp/load_data_txn_error_term.csv" - - fp, err := os.OpenFile(path, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) + fp, err := os.CreateTemp("", "load_data_txn_error_term.csv") require.NoError(t, err) require.NotNil(t, fp) + path := fp.Name() defer func() { _ = os.Remove(path) @@ -591,10 +591,10 @@ func (cli *testServerClient) runTestLoadDataAutoRandomWithSpecialTerm(t *testing } func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { - path := "/tmp/load_data_list_partition.csv" - defer func() { - _ = os.Remove(path) - }() + f, err := os.CreateTemp("", "load_data_list_partition.csv") + require.NoError(t, err) + defer os.Remove(f.Name()) + path := f.Name() cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true @@ -609,20 +609,20 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(t, path, "1 a", "2 b") + cli.prepareLoadDataFile(t, f, "1 a", "2 b") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows := dbt.MustQuery("select * from t partition(p1) order by id") cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. dbt.MustExec("delete from t") - cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + cli.prepareLoadDataFile(t, f, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") require.NoError(t, rows.Close()) // Test load data meet duplicate error. - cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + cli.prepareLoadDataFile(t, f, "1 x", "2 b", "2 x", "7 a") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, @@ -632,7 +632,7 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. - cli.prepareLoadDataFile(t, path, "5 a", "100 x") + cli.prepareLoadDataFile(t, f, "5 a", "100 x") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) rows = dbt.MustQuery("show warnings") @@ -645,10 +645,10 @@ func (cli *testServerClient) runTestLoadDataForListPartition(t *testing.T) { } func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { - path := "/tmp/load_data_list_partition.csv" - defer func() { - _ = os.Remove(path) - }() + f, err := os.CreateTemp("", "load_data_list_partition.csv") + require.NoError(t, err) + defer os.Remove(f.Name()) + path := f.Name() cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true @@ -663,19 +663,19 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(t, path, "1 a", "2 b") + cli.prepareLoadDataFile(t, f, "1 a", "2 b") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) rows := dbt.MustQuery("select id,name from t partition(p1) order by id") cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. dbt.MustExec("delete from t") - cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + cli.prepareLoadDataFile(t, f, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) require.NoError(t, rows.Close()) rows = dbt.MustQuery("select id,name from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") // Test load data meet duplicate error. - cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + cli.prepareLoadDataFile(t, f, "1 x", "2 b", "2 x", "7 a") require.NoError(t, rows.Close()) dbt.MustExec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) rows = dbt.MustQuery("show warnings") @@ -687,7 +687,7 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") require.NoError(t, rows.Close()) // Test load data meet no partition warning. - cli.prepareLoadDataFile(t, path, "5 a", "100 x") + cli.prepareLoadDataFile(t, f, "5 a", "100 x") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t (id,name)", path)) require.NoError(t, err) rows = dbt.MustQuery("show warnings") @@ -700,10 +700,10 @@ func (cli *testServerClient) runTestLoadDataForListPartition2(t *testing.T) { } func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) { - path := "/tmp/load_data_list_partition.csv" - defer func() { - _ = os.Remove(path) - }() + f, err := os.CreateTemp("", "load_data_list_partition.csv") + require.NoError(t, err) + defer os.Remove(f.Name()) + path := f.Name() cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true @@ -718,20 +718,20 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) partition p3 values in (7,8,15,16,null) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(t, path, "1 a", "2 b") + cli.prepareLoadDataFile(t, f, "1 a", "2 b") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows := dbt.MustQuery("select * from t partition(p1) order by id") cli.checkRows(t, rows, "1 a", "2 b") // Test load data into multi-partitions. dbt.MustExec("delete from t") - cli.prepareLoadDataFile(t, path, "1 a", "3 c", "4 e") + cli.prepareLoadDataFile(t, f, "1 a", "3 c", "4 e") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "3 c", "4 e") require.NoError(t, rows.Close()) // Test load data meet duplicate error. - cli.prepareLoadDataFile(t, path, "1 x", "2 b", "2 x", "7 a") + cli.prepareLoadDataFile(t, f, "1 x", "2 b", "2 x", "7 a") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, @@ -741,7 +741,7 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "1 a", "2 b", "3 c", "4 e", "7 a") // Test load data meet no partition warning. - cli.prepareLoadDataFile(t, path, "5 a", "100 x") + cli.prepareLoadDataFile(t, f, "5 a", "100 x") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) require.NoError(t, rows.Close()) @@ -755,10 +755,10 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition(t *testing.T) } func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T) { - path := "/tmp/load_data_list_partition.csv" - defer func() { - _ = os.Remove(path) - }() + f, err := os.CreateTemp("", "load_data_list_partition.csv") + require.NoError(t, err) + defer os.Remove(f.Name()) + path := f.Name() cli.runTestsOnNewDB(t, func(config *mysql.Config) { config.AllowAllFiles = true @@ -772,19 +772,19 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T partition p_south values in (('s',13),('s',14),('s',15),('s',16)) );`) // Test load data into 1 partition. - cli.prepareLoadDataFile(t, path, "w 1 1", "w 2 2") + cli.prepareLoadDataFile(t, f, "w 1 1", "w 2 2") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) rows := dbt.MustQuery("select * from t partition(p_west) order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2") // Test load data into multi-partitions. dbt.MustExec("delete from t") - cli.prepareLoadDataFile(t, path, "w 1 1", "e 5 5", "n 9 9") + cli.prepareLoadDataFile(t, f, "w 1 1", "e 5 5", "n 9 9") dbt.MustExec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, rows.Close()) rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "e 5 5", "n 9 9") // Test load data meet duplicate error. - cli.prepareLoadDataFile(t, path, "w 1 2", "w 2 2") + cli.prepareLoadDataFile(t, f, "w 1 2", "w 2 2") _, err := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) require.NoError(t, rows.Close()) @@ -794,13 +794,13 @@ func (cli *testServerClient) runTestLoadDataForListColumnPartition2(t *testing.T rows = dbt.MustQuery("select * from t order by id") cli.checkRows(t, rows, "w 1 1", "w 2 2", "e 5 5", "n 9 9") // Test load data meet no partition warning. - cli.prepareLoadDataFile(t, path, "w 3 3", "w 5 5", "e 8 8") + cli.prepareLoadDataFile(t, f, "w 3 3", "w 5 5", "e 8 8") _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) require.NoError(t, rows.Close()) rows = dbt.MustQuery("show warnings") cli.checkRows(t, rows, "Warning 1526 Table has no partition for value from column_list") - cli.prepareLoadDataFile(t, path, "x 1 1", "w 1 1") + cli.prepareLoadDataFile(t, f, "x 1 1", "w 1 1") _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table t", path)) require.NoError(t, err) require.NoError(t, rows.Close()) @@ -851,10 +851,9 @@ func (cli *testServerClient) checkRows(t *testing.T, rows *sql.Rows, expectedRow } func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { - // create a file and write data. - path := "/tmp/load_data_test.csv" - fp, err := os.Create(path) + fp, err := os.CreateTemp("", "load_data_test.csv") require.NoError(t, err) + path := fp.Name() require.NotNil(t, fp) defer func() { err = fp.Close() @@ -886,14 +885,14 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { dbt.MustExec("create sequence s1") // can't insert into views (in TiDB) or sequences. issue #20880 - _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table v1") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table v1", path)) require.Error(t, err) require.Equal(t, "Error 1105: can only load data into base tables", err.Error()) - _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table s1") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table s1", path)) require.Error(t, err) require.Equal(t, "Error 1105: can only load data into base tables", err.Error()) - rs, err1 := dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test") + rs, err1 := dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test", path)) require.NoError(t, err1) lastID, err1 := rs.LastInsertId() require.NoError(t, err1) @@ -944,7 +943,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { // specify faileds and lines dbt.MustExec("delete from test") dbt.MustExec("set @@tidb_dml_batch_size = 3") - rs, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") + rs, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'", path)) require.NoError(t, err) lastID, err = rs.LastInsertId() require.NoError(t, err) @@ -988,7 +987,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.NoError(t, err) } dbt.MustExec("set @@tidb_dml_batch_size = 3") - rs, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'") + rs, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test fields terminated by '\t- ' lines starting by 'xxx ' terminated by '\n'", path)) require.NoError(t, err) lastID, err = rs.LastInsertId() require.NoError(t, err) @@ -1001,7 +1000,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.NoError(t, rows.Close()) // don't support lines terminated is "" dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test lines terminated by ''") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test lines terminated by ''", path)) require.NotNil(t, err) // infile doesn't exist @@ -1032,7 +1031,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { }, "LoadData", func(dbt *testkit.DBTestKit) { dbt.MustExec("create table test (str varchar(10) default null, i int default null)") dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table test FIELDS TERMINATED BY ',' enclosed by '"'`, path)) require.NoError(t, err1) var ( str string @@ -1081,7 +1080,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { }, "LoadData", func(dbt *testkit.DBTestKit) { dbt.MustExec("create table test (a date, b date, c date not null, d date)") dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ','`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table test FIELDS TERMINATED BY ','`, path)) require.NoError(t, err1) var ( a sql.NullString @@ -1138,7 +1137,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { }, "LoadData", func(dbt *testkit.DBTestKit) { dbt.MustExec("create table test (a varchar(20), b varchar(20))") dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' enclosed by '"'`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table test FIELDS TERMINATED BY ',' enclosed by '"'`, path)) require.NoError(t, err1) var ( a sql.NullString @@ -1185,7 +1184,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { }, "LoadData", func(dbt *testkit.DBTestKit) { dbt.MustExec("create table test (id INT NOT NULL PRIMARY KEY, b INT, c varchar(10))") dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table test FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '\"' IGNORE 1 LINES`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table test FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '\"' IGNORE 1 LINES`, path)) require.NoError(t, err1) var ( a int @@ -1211,7 +1210,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { }, "LoadData", func(dbt *testkit.DBTestKit) { dbt.MustExec("create table test (a varchar(255), b varchar(255) default 'default value', c int not null auto_increment, primary key(c))") dbt.MustExec("set @@tidb_dml_batch_size = 3") - _, err = dbt.GetDB().Exec("load data local infile '/tmp/load_data_test.csv' into table test") + _, err = dbt.GetDB().Exec(fmt.Sprintf("load data local infile %q into table test", path)) require.Error(t, err) checkErrorCode(t, err, errno.ErrNotAllowedCommand) }) @@ -1239,7 +1238,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { dbt.MustExec("drop table if exists pn") dbt.MustExec("create table pn (c1 int, c2 int)") dbt.MustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table pn FIELDS TERMINATED BY ','`, path)) require.NoError(t, err1) var ( a int @@ -1260,7 +1259,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { require.NoError(t, rows.Close()) // fail error processing test require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/commitOneTaskErr", "return")) - _, err1 = dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ','`) + _, err1 = dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table pn FIELDS TERMINATED BY ','`, path)) mysqlErr, ok := err1.(*mysql.MySQLError) require.True(t, ok) require.Equal(t, "mock commit one task error", mysqlErr.Message) @@ -1291,7 +1290,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { dbt.MustExec("drop table if exists pn") dbt.MustExec("create table pn (c1 int, c2 int)") dbt.MustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, c2)`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table pn FIELDS TERMINATED BY ',' (c1, c2)`, path)) require.NoError(t, err1) var ( a int @@ -1335,7 +1334,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { dbt.MustExec("drop table if exists pn") dbt.MustExec("create table pn (c1 int, c2 int, c3 int)") dbt.MustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @dummy)`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table pn FIELDS TERMINATED BY ',' (c1, @dummy)`, path)) require.NoError(t, err1) var ( a int @@ -1382,7 +1381,7 @@ func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { dbt.MustExec("drop table if exists pn") dbt.MustExec("create table pn (c1 int, c2 int, c3 int)") dbt.MustExec("set @@tidb_dml_batch_size = 1") - _, err1 := dbt.GetDB().Exec(`load data local infile '/tmp/load_data_test.csv' into table pn FIELDS TERMINATED BY ',' (c1, @val1, @val2) SET c3 = @val2 * 100, c2 = CAST(@val1 AS UNSIGNED)`) + _, err1 := dbt.GetDB().Exec(fmt.Sprintf(`load data local infile %q into table pn FIELDS TERMINATED BY ',' (c1, @val1, @val2) SET c3 = @val2 * 100, c2 = CAST(@val1 AS UNSIGNED)`, path)) require.NoError(t, err1) var ( a int diff --git a/server/tidb_serial_test.go b/server/tidb_serial_test.go index 5bbf88ad0e392..39907170ade80 100644 --- a/server/tidb_serial_test.go +++ b/server/tidb_serial_test.go @@ -20,6 +20,7 @@ import ( "context" "crypto/x509" "os" + "path/filepath" "sync/atomic" "testing" "time" @@ -31,13 +32,12 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) // this test will change `kv.TxnTotalSizeLimit` which may affect other test suites, // so we must make it running in serial. -func TestLoadData(t *testing.T) { +func TestLoadData1(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() @@ -130,31 +130,22 @@ func TestTLSBasic(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() + dir := t.TempDir() + + fileName := func(file string) string { + return filepath.Join(dir, file) + } + // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, fileName("ca-key.pem"), fileName("ca-cert.pem")) require.NoError(t, err) - serverCert, _, err := generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") + serverCert, _, err := generateCert(1, "tidb-server", caCert, caKey, fileName("server-key.pem"), fileName("server-cert.pem")) require.NoError(t, err) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, fileName("client-key.pem"), fileName("client-cert.pem")) require.NoError(t, err) - err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) + err = registerTLSConfig("client-certificate", fileName("ca-cert.pem"), fileName("client-cert.pem"), fileName("client-key.pem"), "tidb-server", true) require.NoError(t, err) - defer func() { - err := os.Remove("/tmp/ca-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/ca-cert.pem") - require.NoError(t, err) - err = os.Remove("/tmp/server-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/server-cert.pem") - require.NoError(t, err) - err = os.Remove("/tmp/client-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/client-cert.pem") - require.NoError(t, err) - }() - // Start the server with TLS but without CA, in this case the server will not verify client's certificate. connOverrider := func(config *mysql.Config) { config.TLSConfig = "skip-verify" @@ -164,8 +155,8 @@ func TestTLSBasic(t *testing.T) { cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security = config.Security{ - SSLCert: "/tmp/server-cert.pem", - SSLKey: "/tmp/server-key.pem", + SSLCert: fileName("server-cert.pem"), + SSLKey: fileName("server-key.pem"), } server, err := NewServer(cfg, ts.tidbdrv) require.NoError(t, err) @@ -204,40 +195,31 @@ func TestTLSVerify(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() + dir := t.TempDir() + + fileName := func(file string) string { + return filepath.Join(dir, file) + } + // Generate valid TLS certificates. - caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") + caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, fileName("ca-key.pem"), fileName("ca-cert.pem")) require.NoError(t, err) - _, _, err = generateCert(1, "tidb-server", caCert, caKey, "/tmp/server-key.pem", "/tmp/server-cert.pem") + _, _, err = generateCert(1, "tidb-server", caCert, caKey, fileName("server-key.pem"), fileName("server-cert.pem")) require.NoError(t, err) - _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, "/tmp/client-key.pem", "/tmp/client-cert.pem") + _, _, err = generateCert(2, "SQL Client Certificate", caCert, caKey, fileName("client-key.pem"), fileName("client-cert.pem")) require.NoError(t, err) - err = registerTLSConfig("client-certificate", "/tmp/ca-cert.pem", "/tmp/client-cert.pem", "/tmp/client-key.pem", "tidb-server", true) + err = registerTLSConfig("client-certificate", fileName("ca-cert.pem"), fileName("client-cert.pem"), fileName("client-key.pem"), "tidb-server", true) require.NoError(t, err) - defer func() { - err := os.Remove("/tmp/ca-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/ca-cert.pem") - require.NoError(t, err) - err = os.Remove("/tmp/server-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/server-cert.pem") - require.NoError(t, err) - err = os.Remove("/tmp/client-key.pem") - require.NoError(t, err) - err = os.Remove("/tmp/client-cert.pem") - require.NoError(t, err) - }() - // Start the server with TLS & CA, if the client presents its certificate, the certificate will be verified. cli := newTestServerClient() cfg := newTestConfig() cfg.Port = cli.port cfg.Status.ReportStatus = false cfg.Security = config.Security{ - SSLCA: "/tmp/ca-cert.pem", - SSLCert: "/tmp/server-cert.pem", - SSLKey: "/tmp/server-key.pem", + SSLCA: fileName("ca-cert.pem"), + SSLCert: fileName("server-cert.pem"), + SSLKey: fileName("server-key.pem"), } server, err := NewServer(cfg, ts.tidbdrv) require.NoError(t, err) @@ -269,7 +251,7 @@ func TestTLSVerify(t *testing.T) { _, _, err = util.LoadTLSCertificates("", "wrong key", "wrong cert", true, 528) require.Error(t, err) - _, _, err = util.LoadTLSCertificates("wrong ca", "/tmp/server-key.pem", "/tmp/server-cert.pem", true, 528) + _, _, err = util.LoadTLSCertificates("wrong ca", fileName("server-key.pem"), fileName("server-cert.pem"), true, 528) require.Error(t, err) } @@ -371,8 +353,6 @@ func TestDefaultCharacterAndCollation(t *testing.T) { defer cleanup() // issue #21194 - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) // 255 is the collation id of mysql client 8 default collation_connection qctx, err := ts.tidbdrv.OpenCtx(uint64(0), 0, uint8(255), "test", nil) require.NoError(t, err) diff --git a/server/tidb_test.go b/server/tidb_test.go index d07d8ca5f888b..77f7f8317afc5 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -248,26 +248,25 @@ func TestStatusAPIWithTLS(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() - caCert, caKey, err := generateCert(0, "TiDB CA 2", nil, nil, "/tmp/ca-key-2.pem", "/tmp/ca-cert-2.pem") + dir := t.TempDir() + + fileName := func(file string) string { + return filepath.Join(dir, file) + } + + caCert, caKey, err := generateCert(0, "TiDB CA 2", nil, nil, fileName("ca-key-2.pem"), fileName("ca-cert-2.pem")) require.NoError(t, err) - _, _, err = generateCert(1, "tidb-server-2", caCert, caKey, "/tmp/server-key-2.pem", "/tmp/server-cert-2.pem") + _, _, err = generateCert(1, "tidb-server-2", caCert, caKey, fileName("server-key-2.pem"), fileName("server-cert-2.pem")) require.NoError(t, err) - defer func() { - os.Remove("/tmp/ca-key-2.pem") - os.Remove("/tmp/ca-cert-2.pem") - os.Remove("/tmp/server-key-2.pem") - os.Remove("/tmp/server-cert-2.pem") - }() - cli := newTestServerClient() cli.statusScheme = "https" cfg := newTestConfig() cfg.Port = cli.port cfg.Status.StatusPort = cli.statusPort - cfg.Security.ClusterSSLCA = "/tmp/ca-cert-2.pem" - cfg.Security.ClusterSSLCert = "/tmp/server-cert-2.pem" - cfg.Security.ClusterSSLKey = "/tmp/server-key-2.pem" + cfg.Security.ClusterSSLCA = fileName("ca-cert-2.pem") + cfg.Security.ClusterSSLCert = fileName("server-cert-2.pem") + cfg.Security.ClusterSSLKey = fileName("server-key-2.pem") server, err := NewServer(cfg, ts.tidbdrv) require.NoError(t, err) cli.port = getPortFromTCPAddr(server.listener.Addr()) @@ -293,15 +292,17 @@ func TestStatusAPIWithTLSCNCheck(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() - caPath := filepath.Join(os.TempDir(), "ca-cert-cn.pem") - serverKeyPath := filepath.Join(os.TempDir(), "server-key-cn.pem") - serverCertPath := filepath.Join(os.TempDir(), "server-cert-cn.pem") - client1KeyPath := filepath.Join(os.TempDir(), "client-key-cn-check-a.pem") - client1CertPath := filepath.Join(os.TempDir(), "client-cert-cn-check-a.pem") - client2KeyPath := filepath.Join(os.TempDir(), "client-key-cn-check-b.pem") - client2CertPath := filepath.Join(os.TempDir(), "client-cert-cn-check-b.pem") + dir := t.TempDir() - caCert, caKey, err := generateCert(0, "TiDB CA CN CHECK", nil, nil, filepath.Join(os.TempDir(), "ca-key-cn.pem"), caPath) + caPath := filepath.Join(dir, "ca-cert-cn.pem") + serverKeyPath := filepath.Join(dir, "server-key-cn.pem") + serverCertPath := filepath.Join(dir, "server-cert-cn.pem") + client1KeyPath := filepath.Join(dir, "client-key-cn-check-a.pem") + client1CertPath := filepath.Join(dir, "client-cert-cn-check-a.pem") + client2KeyPath := filepath.Join(dir, "client-key-cn-check-b.pem") + client2CertPath := filepath.Join(dir, "client-cert-cn-check-b.pem") + + caCert, caKey, err := generateCert(0, "TiDB CA CN CHECK", nil, nil, filepath.Join(dir, "ca-key-cn.pem"), caPath) require.NoError(t, err) _, _, err = generateCert(1, "tidb-server-cn-check", caCert, caKey, serverKeyPath, serverCertPath) require.NoError(t, err) @@ -376,11 +377,8 @@ func TestMultiStatements(t *testing.T) { } func TestSocketForwarding(t *testing.T) { - osTempDir := os.TempDir() - tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - require.NoError(t, err) + tempDir := t.TempDir() socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK - defer os.RemoveAll(tempDir) ts, cleanup := createTidbTestSuite(t) defer cleanup() @@ -412,11 +410,8 @@ func TestSocketForwarding(t *testing.T) { } func TestSocket(t *testing.T) { - osTempDir := os.TempDir() - tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - require.NoError(t, err) + tempDir := t.TempDir() socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK - defer os.RemoveAll(tempDir) cfg := newTestConfig() cfg.Socket = socketFile @@ -451,11 +446,8 @@ func TestSocket(t *testing.T) { } func TestSocketAndIp(t *testing.T) { - osTempDir := os.TempDir() - tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - require.NoError(t, err) + tempDir := t.TempDir() socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK - defer os.RemoveAll(tempDir) cli := newTestServerClient() cfg := newTestConfig() @@ -620,11 +612,8 @@ func TestSocketAndIp(t *testing.T) { // TestOnlySocket for server configuration without network interface for mysql clients func TestOnlySocket(t *testing.T) { - osTempDir := os.TempDir() - tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - require.NoError(t, err) + tempDir := t.TempDir() socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK - defer os.RemoveAll(tempDir) cli := newTestServerClient() cfg := newTestConfig() @@ -1794,11 +1783,8 @@ func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, t *testing.T, fn fu } func TestLocalhostClientMapping(t *testing.T) { - osTempDir := os.TempDir() - tempDir, err := os.MkdirTemp(osTempDir, "tidb-test.*.socket") - require.NoError(t, err) + tempDir := t.TempDir() socketFile := tempDir + "/tidbtest.sock" // Unix Socket does not work on Windows, so '/' should be OK - defer os.RemoveAll(tempDir) cli := newTestServerClient() cfg := newTestConfig() diff --git a/session/bootstrap.go b/session/bootstrap.go index f408b8ca0a90e..214ba3174564d 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1864,6 +1864,12 @@ func doDMLWorks(s Session) { if v.Name == variable.TiDBEnable1PC && config.GetGlobalConfig().Store == "tikv" { vVal = variable.On } + if v.Name == variable.TiDBEnableMutationChecker { + vVal = variable.On + } + if v.Name == variable.TiDBTxnAssertionLevel { + vVal = variable.AssertionFastStr + } value := fmt.Sprintf(`("%s", "%s")`, strings.ToLower(k), vVal) values = append(values, value) } diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index e1921df899f2c..338cb0cb9a793 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -218,7 +218,7 @@ func TestUpgrade(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() se := createSessionAndSetID(t, store) @@ -264,11 +264,10 @@ func TestUpgrade(t *testing.T) { ver, err = getBootstrapVersion(se1) require.NoError(t, err) require.Equal(t, int64(0), ver) - + dom.Close() // Create a new session then upgrade() will run automatically. - dom, err := BootstrapSession(store) + dom, err = BootstrapSession(store) require.NoError(t, err) - defer dom.Close() se2 := createSessionAndSetID(t, store) r = mustExec(t, se2, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version"`) @@ -293,6 +292,7 @@ func TestUpgrade(t *testing.T) { require.Equal(t, 1, req.NumRows()) require.Equal(t, "False", req.GetRow(0).GetString(0)) require.NoError(t, r.Close()) + dom.Close() } func TestIssue17979_1(t *testing.T) { @@ -304,9 +304,8 @@ func TestIssue17979_1(t *testing.T) { }() ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() - // test issue 20900, upgrade from v3.0 to v4.0.11+ seV3 := createSessionAndSetID(t, store) txn, err := store.Begin() @@ -323,10 +322,9 @@ func TestIssue17979_1(t *testing.T) { ver, err := getBootstrapVersion(seV3) require.NoError(t, err) require.Equal(t, int64(58), ver) - + dom.Close() domV4, err := BootstrapSession(store) require.NoError(t, err) - defer domV4.Close() seV4 := createSessionAndSetID(t, store) ver, err = getBootstrapVersion(seV4) require.NoError(t, err) @@ -336,6 +334,7 @@ func TestIssue17979_1(t *testing.T) { require.NoError(t, r.Next(ctx, req)) require.Equal(t, "log", req.GetRow(0).GetString(0)) require.Equal(t, config.OOMActionLog, config.GetGlobalConfig().OOMAction) + domV4.Close() } func TestIssue17979_2(t *testing.T) { @@ -347,7 +346,7 @@ func TestIssue17979_2(t *testing.T) { }() ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() // test issue 20900, upgrade from v4.0.11 to v4.0.11 @@ -366,7 +365,7 @@ func TestIssue17979_2(t *testing.T) { ver, err := getBootstrapVersion(seV3) require.NoError(t, err) require.Equal(t, int64(59), ver) - + dom.Close() domV4, err := BootstrapSession(store) require.NoError(t, err) defer domV4.Close() @@ -391,9 +390,8 @@ func TestIssue20900_1(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() - // test issue 20900, upgrade from v3.0 to v4.0.9+ seV3 := createSessionAndSetID(t, store) txn, err := store.Begin() @@ -410,7 +408,7 @@ func TestIssue20900_1(t *testing.T) { ver, err := getBootstrapVersion(seV3) require.NoError(t, err) require.Equal(t, int64(38), ver) - + dom.Close() domV4, err := BootstrapSession(store) require.NoError(t, err) defer domV4.Close() @@ -439,7 +437,7 @@ func TestIssue20900_2(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() // test issue 20900, upgrade from v4.0.8 to v4.0.9+ @@ -458,10 +456,9 @@ func TestIssue20900_2(t *testing.T) { ver, err := getBootstrapVersion(seV3) require.NoError(t, err) require.Equal(t, int64(52), ver) - + dom.Close() domV4, err := BootstrapSession(store) require.NoError(t, err) - defer domV4.Close() seV4 := createSessionAndSetID(t, store) ver, err = getBootstrapVersion(seV4) require.NoError(t, err) @@ -475,6 +472,7 @@ func TestIssue20900_2(t *testing.T) { req = r.NewChunk(nil) require.NoError(t, r.Next(ctx, req)) require.Equal(t, 0, req.NumRows()) + domV4.Close() } func TestANSISQLMode(t *testing.T) { @@ -660,7 +658,7 @@ func TestUpdateDuplicateBindInfo(t *testing.T) { } func TestUpgradeClusteredIndexDefaultValue(t *testing.T) { - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() seV67 := createSessionAndSetID(t, store) @@ -679,10 +677,10 @@ func TestUpgradeClusteredIndexDefaultValue(t *testing.T) { ver, err := getBootstrapVersion(seV67) require.NoError(t, err) require.Equal(t, int64(67), ver) + dom.Close() domV68, err := BootstrapSession(store) require.NoError(t, err) - defer domV68.Close() seV68 := createSessionAndSetID(t, store) ver, err = getBootstrapVersion(seV68) require.NoError(t, err) @@ -695,13 +693,13 @@ func TestUpgradeClusteredIndexDefaultValue(t *testing.T) { row := req.GetRow(0) require.Equal(t, "INT_ONLY", row.GetString(0)) require.Equal(t, "INT_ONLY", row.GetString(1)) + domV68.Close() } func TestUpgradeVersion66(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() - seV65 := createSessionAndSetID(t, store) txn, err := store.Begin() require.NoError(t, err) @@ -717,10 +715,10 @@ func TestUpgradeVersion66(t *testing.T) { ver, err := getBootstrapVersion(seV65) require.NoError(t, err) require.Equal(t, int64(65), ver) - + dom.Close() domV66, err := BootstrapSession(store) require.NoError(t, err) - defer domV66.Close() + seV66 := createSessionAndSetID(t, store) ver, err = getBootstrapVersion(seV66) require.NoError(t, err) @@ -732,6 +730,7 @@ func TestUpgradeVersion66(t *testing.T) { row := req.GetRow(0) require.Equal(t, int64(1), row.GetInt64(0)) require.Equal(t, int64(1), row.GetInt64(1)) + domV66.Close() } func TestUpgradeVersion74(t *testing.T) { @@ -748,7 +747,7 @@ func TestUpgradeVersion74(t *testing.T) { for _, ca := range cases { func() { - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() seV73 := createSessionAndSetID(t, store) @@ -766,7 +765,7 @@ func TestUpgradeVersion74(t *testing.T) { ver, err := getBootstrapVersion(seV73) require.NoError(t, err) require.Equal(t, int64(72), ver) - + dom.Close() domV74, err := BootstrapSession(store) require.NoError(t, err) defer domV74.Close() @@ -787,7 +786,7 @@ func TestUpgradeVersion74(t *testing.T) { func TestUpgradeVersion75(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() seV74 := createSessionAndSetID(t, store) @@ -813,7 +812,7 @@ func TestUpgradeVersion75(t *testing.T) { require.NoError(t, r.Next(ctx, req)) require.Equal(t, "host", strings.ToLower(row.GetString(0))) require.Equal(t, "char(64)", strings.ToLower(row.GetString(1))) - + dom.Close() domV75, err := BootstrapSession(store) require.NoError(t, err) defer domV75.Close() @@ -838,19 +837,16 @@ func TestForIssue23387(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) defer func() { require.NoError(t, store.Close()) }() - _, err = BootstrapSession(store) - // domain leaked here, Close() is not called. For testing, it's OK. - // If we close it and BootstrapSession again, we'll get an error "session pool is closed". - // The problem is caused by some the global level variable, domain map is not intended for multiple instances. + dom, err := BootstrapSession(store) require.NoError(t, err) se := createSessionAndSetID(t, store) se.Auth(&auth.UserIdentity{Username: "root", Hostname: `%`}, nil, []byte("012345678901234567890")) mustExec(t, se, "create user quatest") - + dom.Close() // Upgrade to a newer version, check the user's privilege. currentBootstrapVersion = saveCurrentBootstrapVersion - dom, err := BootstrapSession(store) + dom, err = BootstrapSession(store) require.NoError(t, err) defer dom.Close() @@ -884,7 +880,7 @@ func TestReferencesPrivilegeOnColumn(t *testing.T) { func TestAnalyzeVersionUpgradeFrom300To500(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() // Upgrade from 3.0.0 to 5.1+ or above. @@ -911,7 +907,7 @@ func TestAnalyzeVersionUpgradeFrom300To500(t *testing.T) { err = res.Next(ctx, chk) require.NoError(t, err) require.Equal(t, 0, chk.NumRows()) - + dom.Close() domCurVer, err := BootstrapSession(store) require.NoError(t, err) defer domCurVer.Close() @@ -959,7 +955,7 @@ func TestIndexMergeInNewCluster(t *testing.T) { func TestIndexMergeUpgradeFrom300To540(t *testing.T) { ctx := context.Background() - store, _ := createStoreAndBootstrap(t) + store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() // Upgrade from 3.0.0 to 5.4+. @@ -986,7 +982,7 @@ func TestIndexMergeUpgradeFrom300To540(t *testing.T) { err = res.Next(ctx, chk) require.NoError(t, err) require.Equal(t, 0, chk.NumRows()) - + dom.Close() domCurVer, err := BootstrapSession(store) require.NoError(t, err) defer domCurVer.Close() @@ -1008,66 +1004,68 @@ func TestIndexMergeUpgradeFrom300To540(t *testing.T) { func TestIndexMergeUpgradeFrom400To540(t *testing.T) { for i := 0; i < 2; i++ { - ctx := context.Background() - store, _ := createStoreAndBootstrap(t) - defer func() { require.NoError(t, store.Close()) }() - - // upgrade from 4.0.0 to 5.4+. - ver400 := 46 - seV4 := createSessionAndSetID(t, store) - txn, err := store.Begin() - require.NoError(t, err) - m := meta.NewMeta(txn) - err = m.FinishBootstrap(int64(ver400)) - require.NoError(t, err) - err = txn.Commit(context.Background()) - require.NoError(t, err) - mustExec(t, seV4, fmt.Sprintf("update mysql.tidb set variable_value=%d where variable_name='tidb_server_version'", ver400)) - mustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", variable.Off, variable.TiDBEnableIndexMerge)) - mustExec(t, seV4, "commit") - unsetStoreBootstrapped(store.UUID()) - ver, err := getBootstrapVersion(seV4) - require.NoError(t, err) - require.Equal(t, int64(ver400), ver) + func() { + ctx := context.Background() + store, dom := createStoreAndBootstrap(t) + defer func() { require.NoError(t, store.Close()) }() - // We are now in 4.0.0, tidb_enable_index_merge is off. - res := mustExec(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) - chk := res.NewChunk(nil) - err = res.Next(ctx, chk) - require.NoError(t, err) - require.Equal(t, 1, chk.NumRows()) - row := chk.GetRow(0) - require.Equal(t, 2, row.Len()) - require.Equal(t, variable.Off, row.GetString(1)) - - if i == 0 { - // For the first time, We set tidb_enable_index_merge as on. - // And after upgrade to 5.x, tidb_enable_index_merge should remains to be on. - // For the second it should be off. - mustExec(t, seV4, "set global tidb_enable_index_merge = on") - } + // upgrade from 4.0.0 to 5.4+. + ver400 := 46 + seV4 := createSessionAndSetID(t, store) + txn, err := store.Begin() + require.NoError(t, err) + m := meta.NewMeta(txn) + err = m.FinishBootstrap(int64(ver400)) + require.NoError(t, err) + err = txn.Commit(context.Background()) + require.NoError(t, err) + mustExec(t, seV4, fmt.Sprintf("update mysql.tidb set variable_value=%d where variable_name='tidb_server_version'", ver400)) + mustExec(t, seV4, fmt.Sprintf("update mysql.GLOBAL_VARIABLES set variable_value='%s' where variable_name='%s'", variable.Off, variable.TiDBEnableIndexMerge)) + mustExec(t, seV4, "commit") + unsetStoreBootstrapped(store.UUID()) + ver, err := getBootstrapVersion(seV4) + require.NoError(t, err) + require.Equal(t, int64(ver400), ver) - // Upgrade to 5.x. - domCurVer, err := BootstrapSession(store) - require.NoError(t, err) - defer domCurVer.Close() - seCurVer := createSessionAndSetID(t, store) - ver, err = getBootstrapVersion(seCurVer) - require.NoError(t, err) - require.Equal(t, currentBootstrapVersion, ver) + // We are now in 4.0.0, tidb_enable_index_merge is off. + res := mustExec(t, seV4, fmt.Sprintf("select * from mysql.GLOBAL_VARIABLES where variable_name='%s'", variable.TiDBEnableIndexMerge)) + chk := res.NewChunk(nil) + err = res.Next(ctx, chk) + require.NoError(t, err) + require.Equal(t, 1, chk.NumRows()) + row := chk.GetRow(0) + require.Equal(t, 2, row.Len()) + require.Equal(t, variable.Off, row.GetString(1)) + + if i == 0 { + // For the first time, We set tidb_enable_index_merge as on. + // And after upgrade to 5.x, tidb_enable_index_merge should remains to be on. + // For the second it should be off. + mustExec(t, seV4, "set global tidb_enable_index_merge = on") + } + dom.Close() + // Upgrade to 5.x. + domCurVer, err := BootstrapSession(store) + require.NoError(t, err) + defer domCurVer.Close() + seCurVer := createSessionAndSetID(t, store) + ver, err = getBootstrapVersion(seCurVer) + require.NoError(t, err) + require.Equal(t, currentBootstrapVersion, ver) - // We are now in 5.x, tidb_enable_index_merge should be on because we enable it in 4.0.0. - res = mustExec(t, seCurVer, "select @@tidb_enable_index_merge") - chk = res.NewChunk(nil) - err = res.Next(ctx, chk) - require.NoError(t, err) - require.Equal(t, 1, chk.NumRows()) - row = chk.GetRow(0) - require.Equal(t, 1, row.Len()) - if i == 0 { - require.Equal(t, int64(1), row.GetInt64(0)) - } else { - require.Equal(t, int64(0), row.GetInt64(0)) - } + // We are now in 5.x, tidb_enable_index_merge should be on because we enable it in 4.0.0. + res = mustExec(t, seCurVer, "select @@tidb_enable_index_merge") + chk = res.NewChunk(nil) + err = res.Next(ctx, chk) + require.NoError(t, err) + require.Equal(t, 1, chk.NumRows()) + row = chk.GetRow(0) + require.Equal(t, 1, row.Len()) + if i == 0 { + require.Equal(t, int64(1), row.GetInt64(0)) + } else { + require.Equal(t, int64(0), row.GetInt64(0)) + } + }() } } diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index 32de5a4348632..9aca5077c30ea 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -676,9 +676,6 @@ func TestPartitionTable(t *testing.T) { // https://github.com/pingcap/tidb/issues/23106 func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -702,9 +699,6 @@ func TestClusteredIndexDecodeRestoredDataV5(t *testing.T) { // https://github.com/pingcap/tidb/issues/23178 func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - store, clean := testkit.CreateMockStore(t) defer clean() @@ -724,9 +718,6 @@ func TestPrefixedClusteredIndexUniqueKeyWithNewCollation(t *testing.T) { func TestClusteredIndexNewCollationWithOldRowFormat(t *testing.T) { // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) // but unistore doesn't support old row format. - defer collate.SetNewCollationEnabledForTest(false) - collate.SetNewCollationEnabledForTest(true) - store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/session/main_test.go b/session/main_test.go index fb75aaccf0f3a..ec0cb1ede5d81 100644 --- a/session/main_test.go +++ b/session/main_test.go @@ -56,7 +56,8 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ // TODO: figure the reason and shorten this list goleak.IgnoreTopFunction("github.com/tikv/client-go/v2/internal/retry.newBackoffFn.func1"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/v3.waitRetryBackoff"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 9cd82abe32151..14636177b7153 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -61,14 +61,20 @@ func (s *testPessimisticSuite) new1PCTestKitWithInit(c *C) *testkit.TestKit { return tk } +type lockTTL uint64 + +func setLockTTL(v uint64) lockTTL { return lockTTL(atomic.SwapUint64(&transaction.ManagedLockTTL, v)) } + +func (v lockTTL) restore() { atomic.StoreUint64(&transaction.ManagedLockTTL, uint64(v)) } + type testPessimisticSuite struct { testSessionSuiteBase } func (s *testPessimisticSuite) SetUpSuite(c *C) { s.testSessionSuiteBase.SetUpSuite(c) - // Set it to 300ms for testing lock resolve. - atomic.StoreUint64(&transaction.ManagedLockTTL, 300) + // Set it to 5s for testing lock resolve. + atomic.StoreUint64(&transaction.ManagedLockTTL, 5000) transaction.PrewriteMaxBackoff = 500 } @@ -437,11 +443,6 @@ func (s *testPessimisticSuite) TestLockUnchangedRowKey(c *C) { } func (s *testPessimisticSuite) TestOptimisticConflicts(c *C) { - // To avoid the resolve lock request arrives earlier before heartbeat request while lock expires. - atomic.StoreUint64(&transaction.ManagedLockTTL, 1000) - defer func() { - atomic.StoreUint64(&transaction.ManagedLockTTL, 300) - }() tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists conflict") @@ -659,6 +660,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { func (s *testPessimisticSuite) TestKillStopTTLManager(c *C) { // Test killing an idle pessimistic session stop its ttlManager. + defer setLockTTL(300).restore() tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists test_kill") @@ -673,7 +675,9 @@ func (s *testPessimisticSuite) TestKillStopTTLManager(c *C) { // This query should success rather than returning a ResolveLock error. tk2.MustExec("update test_kill set c = c + 1 where id = 1") - + succ = atomic.CompareAndSwapUint32(&sessVars.Killed, 1, 0) + c.Assert(succ, IsTrue) + tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -717,11 +721,6 @@ func (s *testPessimisticSuite) TestConcurrentInsert(c *C) { } func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { - // Increasing the ManagedLockTTL so that the lock may not be resolved testing with TiKV. - atomic.StoreUint64(&transaction.ManagedLockTTL, 5000) - defer func() { - atomic.StoreUint64(&transaction.ManagedLockTTL, 300) - }() tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists tk") tk.MustExec("create table tk (c1 int primary key, c2 int)") @@ -846,11 +845,6 @@ func (s *testPessimisticSuite) TestPushConditionCheckForPessimisticTxn(c *C) { } func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { - // Increasing the ManagedLockTTL so that the lock may not be resolved testing with TiKV. - atomic.StoreUint64(&transaction.ManagedLockTTL, 5000) - defer func() { - atomic.StoreUint64(&transaction.ManagedLockTTL, 300) - }() // prepare work tk := testkit.NewTestKitWithInit(c, s.store) defer tk.MustExec("drop table if exists tk") @@ -1199,9 +1193,7 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { } func (s *testPessimisticSuite) TestPessimisticCommitReadLock(c *C) { - // set lock ttl to 3s, tk1 lock wait timeout is 2s - atomic.StoreUint64(&transaction.ManagedLockTTL, 3000) - defer atomic.StoreUint64(&transaction.ManagedLockTTL, 300) + // tk1 lock wait timeout is 2s tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test") tk1 := testkit.NewTestKitWithInit(c, s.store) @@ -1305,8 +1297,6 @@ func (s *testPessimisticSuite) TestNonAutoCommitWithPessimisticMode(c *C) { } func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { - atomic.StoreUint64(&transaction.ManagedLockTTL, 3000) - defer atomic.StoreUint64(&transaction.ManagedLockTTL, 300) tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk2.MustExec("use test") @@ -1453,8 +1443,6 @@ func (s *testPessimisticSuite) TestRCIndexMerge(c *C) { } func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { - atomic.StoreUint64(&transaction.ManagedLockTTL, 3000) - defer atomic.StoreUint64(&transaction.ManagedLockTTL, 300) tk := testkit.NewTestKitWithInit(c, s.store) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_row_format_version = %d", variable.DefTiDBRowFormatV2)) @@ -1527,6 +1515,7 @@ func (s *testPessimisticSuite) TestTxnWithExpiredPessimisticLocks(c *C) { func (s *testPessimisticSuite) TestKillWaitLockTxn(c *C) { // Test kill command works on waiting pessimistic lock. + defer setLockTTL(300).restore() tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists test_kill") @@ -2371,7 +2360,7 @@ func (s *testPessimisticSuite) TestIssue21498(c *C) { tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1") for _, partition := range []bool{false, true} { - //RC test + // RC test tk.MustExec("drop table if exists t, t1") createTable := "create table t (id int primary key, v int, index iv (v))" if partition { @@ -2540,7 +2529,7 @@ func (s *testPessimisticSuite) TestPlanCacheSchemaChange(c *C) { tk.MustExec("set tidb_enable_amend_pessimistic_txn = 1") tk2.MustExec("set tidb_enable_amend_pessimistic_txn = 1") - //generate plan cache + // generate plan cache tk.MustExec("prepare update_stmt from 'update t set vv = vv + 1 where v = ?'") tk.MustExec("set @v = 1") tk.MustExec("execute update_stmt using @v") @@ -2582,10 +2571,6 @@ func (s *testPessimisticSuite) TestPlanCacheSchemaChange(c *C) { } func (s *testPessimisticSuite) TestAsyncCommitCalTSFail(c *C) { - atomic.StoreUint64(&transaction.ManagedLockTTL, 5000) - defer func() { - atomic.StoreUint64(&transaction.ManagedLockTTL, 300) - }() defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.TiKVClient.AsyncCommit.SafeWindow = time.Second @@ -2851,3 +2836,29 @@ func (s *testPessimisticSuite) TestAmendForColumnChange(c *C) { tk2.MustExec("drop database test_db") } + +func (s *testPessimisticSuite) TestPessimisticAutoCommitTxn(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_txn_mode = 'pessimistic'") + tk.MustExec("drop database if exists test_db") + tk.MustExec("create database test_db") + tk.MustExec("use test_db") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (i int)") + tk.MustExec("insert into t values (1)") + tk.MustExec("set autocommit = on") + + rows := tk.MustQuery("explain update t set i = -i").Rows() + explain := fmt.Sprintf("%v", rows[1]) + c.Assert(explain, Not(Matches), ".*SelectLock.*") + + originCfg := config.GetGlobalConfig() + defer config.StoreGlobalConfig(originCfg) + newCfg := *originCfg + newCfg.PessimisticTxn.PessimisticAutoCommit.Store(true) + config.StoreGlobalConfig(&newCfg) + + rows = tk.MustQuery("explain update t set i = -i").Rows() + explain = fmt.Sprintf("%v", rows[1]) + c.Assert(explain, Matches, ".*SelectLock.*") +} diff --git a/session/schema_amender.go b/session/schema_amender.go index 8993190d2fef5..9275ee4ae08d7 100644 --- a/session/schema_amender.go +++ b/session/schema_amender.go @@ -393,7 +393,8 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc for i := 0; i < len(deletedMutations.GetKeys()); i++ { key := deletedMutations.GetKeys()[i] if _, ok := a.insertedNewIndexKeys[string(key)]; !ok { - resAddMutations.Push(deletedMutations.GetOps()[i], key, deletedMutations.GetValues()[i], deletedMutations.GetPessimisticFlags()[i]) + resAddMutations.Push(deletedMutations.GetOps()[i], key, deletedMutations.GetValues()[i], deletedMutations.IsPessimisticLock(i), + deletedMutations.IsAssertExists(i), deletedMutations.IsAssertNotExist(i)) } } for i := 0; i < len(insertedMutations.GetKeys()); i++ { @@ -402,7 +403,8 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc if _, ok := a.deletedOldIndexKeys[string(key)]; ok { destKeyOp = kvrpcpb.Op_Put } - resAddMutations.Push(destKeyOp, key, insertedMutations.GetValues()[i], insertedMutations.GetPessimisticFlags()[i]) + resAddMutations.Push(destKeyOp, key, insertedMutations.GetValues()[i], insertedMutations.IsPessimisticLock(i), + insertedMutations.IsAssertExists(i), insertedMutations.IsAssertNotExist(i)) } } else { resAddMutations.MergeMutations(deletedMutations) @@ -492,7 +494,11 @@ func (a *amendOperationAddIndex) genNewIdxKey(ctx context.Context, sctx sessionc isPessimisticLock = true } a.insertedNewIndexKeys[string(newIdxKey)] = struct{}{} - newMutation := &transaction.PlainMutation{KeyOp: newIndexOp, Key: newIdxKey, Value: newIdxValue, IsPessimisticLock: isPessimisticLock} + var flags transaction.CommitterMutationFlags + if isPessimisticLock { + flags |= transaction.MutationFlagIsPessimisticLock + } + newMutation := &transaction.PlainMutation{KeyOp: newIndexOp, Key: newIdxKey, Value: newIdxValue, Flags: flags} return newMutation, nil } @@ -519,7 +525,11 @@ func (a *amendOperationAddIndex) genOldIdxKey(ctx context.Context, sctx sessionc isPessimisticLock = true } a.deletedOldIndexKeys[string(newIdxKey)] = struct{}{} - return &transaction.PlainMutation{KeyOp: kvrpcpb.Op_Del, Key: newIdxKey, Value: emptyVal, IsPessimisticLock: isPessimisticLock}, nil + var flags transaction.CommitterMutationFlags + if isPessimisticLock { + flags |= transaction.MutationFlagIsPessimisticLock + } + return &transaction.PlainMutation{KeyOp: kvrpcpb.Op_Del, Key: newIdxKey, Value: emptyVal, Flags: flags}, nil } return nil, nil } diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index a22f21ad6e225..514c82bd411f1 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -66,7 +66,7 @@ func mutationsEqual(res *transaction.PlainMutations, expected *transaction.Plain } require.GreaterOrEqual(t, foundIdx, 0) require.Equal(t, expected.GetOps()[foundIdx], res.GetOps()[i]) - require.Equal(t, expected.GetPessimisticFlags()[foundIdx], res.GetPessimisticFlags()[i]) + require.Equal(t, expected.IsPessimisticLock(foundIdx), res.IsPessimisticLock(i)) require.Equal(t, expected.GetKeys()[foundIdx], res.GetKeys()[i]) require.Equal(t, expected.GetValues()[foundIdx], res.GetValues()[i]) } @@ -140,7 +140,7 @@ func prepareTestData( oldData.ops = append(oldData.ops, keyOp) oldData.rowValue = append(oldData.rowValue, thisRowValue) if keyOp == kvrpcpb.Op_Del { - mutations.Push(keyOp, rowKey, []byte{}, true) + mutations.Push(keyOp, rowKey, []byte{}, true, false, false) } } oldRowValues[i] = thisRowValue @@ -168,9 +168,9 @@ func prepareTestData( } require.NoError(t, err) if keyOp == kvrpcpb.Op_Put || keyOp == kvrpcpb.Op_Insert { - mutations.Push(keyOp, rowKey, rowValue, true) + mutations.Push(keyOp, rowKey, rowValue, true, false, false) } else if keyOp == kvrpcpb.Op_Lock { - mutations.Push(keyOp, rowKey, []byte{}, true) + mutations.Push(keyOp, rowKey, []byte{}, true, false, false) } newRowValues[i] = thisRowValue newRowKvMap[string(rowKey)] = thisRowValue @@ -209,7 +209,7 @@ func prepareTestData( if info.indexInfoAtCommit.Meta().Unique { isPessimisticLock = true } - oldIdxKeyMutation.Push(kvrpcpb.Op_Del, idxKey, []byte{}, isPessimisticLock) + oldIdxKeyMutation.Push(kvrpcpb.Op_Del, idxKey, []byte{}, isPessimisticLock, false, false) } } if addIndexNeedAddOp(info.AmendOpType) && mayGenPutIndexRowKeyOp(keyOp) { @@ -221,7 +221,7 @@ func prepareTestData( mutOp = kvrpcpb.Op_Insert isPessimisticLock = true } - newIdxKeyMutation.Push(mutOp, idxKey, idxVal, isPessimisticLock) + newIdxKeyMutation.Push(mutOp, idxKey, idxVal, isPessimisticLock, false, false) } skipMerge := false if info.AmendOpType == AmendNeedAddDeleteAndInsert { @@ -436,7 +436,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { idxKey := tablecodec.EncodeIndexSeekKey(oldTbInfo.Meta().ID, oldTbInfo.Indices()[i].Meta().ID, idxValue) err = txn.Set(idxKey, idxValue) require.NoError(t, err) - mutations.Push(kvrpcpb.Op_Put, idxKey, idxValue, false) + mutations.Push(kvrpcpb.Op_Put, idxKey, idxValue, false, false, false) } res, err := schemaAmender.genAllAmendMutations(ctx, &mutations, collector) @@ -446,13 +446,13 @@ func TestAmendCollectAndGenMutations(t *testing.T) { // Validate generated results. require.Equal(t, len(res.GetOps()), len(res.GetKeys())) require.Equal(t, len(res.GetOps()), len(res.GetValues())) - require.Equal(t, len(res.GetOps()), len(res.GetPessimisticFlags())) + require.Equal(t, len(res.GetOps()), len(res.GetFlags())) for i := 0; i < len(expectedMutations.GetKeys()); i++ { logutil.BgLogger().Info("[TEST] expected mutations", zap.Stringer("key", kv.Key(expectedMutations.GetKeys()[i])), zap.Stringer("val", kv.Key(expectedMutations.GetKeys()[i])), zap.Stringer("op_type", expectedMutations.GetOps()[i]), - zap.Bool("is_pessimistic", expectedMutations.GetPessimisticFlags()[i]), + zap.Bool("is_pessimistic", expectedMutations.IsPessimisticLock(i)), ) } for i := 0; i < len(res.GetKeys()); i++ { @@ -460,7 +460,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { zap.Stringer("key", kv.Key(res.GetKeys()[i])), zap.Stringer("val", kv.Key(res.GetKeys()[i])), zap.Stringer("op_type", res.GetOps()[i]), - zap.Bool("is_pessimistic", res.GetPessimisticFlags()[i]), + zap.Bool("is_pessimistic", res.IsPessimisticLock(i)), ) } mutationsEqual(res, &expectedMutations, t) diff --git a/session/session.go b/session/session.go index 58e18b0109da7..7cb5e5001be5b 100644 --- a/session/session.go +++ b/session/session.go @@ -22,7 +22,9 @@ import ( "bytes" "context" "crypto/tls" + "encoding/hex" "encoding/json" + stderrs "errors" "fmt" "runtime/pprof" "runtime/trace" @@ -46,12 +48,15 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/store/driver/txn" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/table/temptable" + "github.com/pingcap/tidb/util/logutil/consistency" "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" "github.com/pingcap/tidb/util/topsql/stmtstats" "github.com/pingcap/tipb/go-binlog" + tikverr "github.com/tikv/client-go/v2/error" "go.uber.org/zap" "github.com/pingcap/tidb/bindinfo" @@ -599,7 +604,11 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(kv.CommitTSUpperBoundCheck, c.commitTSCheck) } - return s.commitTxnWithTemporaryData(tikvutil.SetSessionID(ctx, sessVars.ConnectionID), &s.txn) + err = s.commitTxnWithTemporaryData(tikvutil.SetSessionID(ctx, sessVars.ConnectionID), &s.txn) + if err != nil { + err = s.handleAssertionFailure(ctx, err) + } + return err } type cachedTableRenewLease struct { @@ -685,6 +694,62 @@ func (c *cachedTableRenewLease) commitTSCheck(commitTS uint64) bool { return true } +// handleAssertionFailure extracts the possible underlying assertionFailed error, +// gets the corresponding MVCC history and logs it. +// If it's not an assertion failure, returns the original error. +func (s *session) handleAssertionFailure(ctx context.Context, err error) error { + var assertionFailure *tikverr.ErrAssertionFailed + if !stderrs.As(err, &assertionFailure) { + return err + } + key := assertionFailure.Key + newErr := kv.ErrAssertionFailed.GenWithStackByArgs( + hex.EncodeToString(key), assertionFailure.Assertion.String(), assertionFailure.StartTs, + assertionFailure.ExistingStartTs, assertionFailure.ExistingCommitTs, + ) + + if s.GetSessionVars().EnableRedactLog { + return newErr + } + + var decodeFunc func(kv.Key, *kvrpcpb.MvccGetByKeyResponse, map[string]interface{}) + // if it's a record key or an index key, decode it + if infoSchema, ok := s.sessionVars.TxnCtx.InfoSchema.(infoschema.InfoSchema); ok && + infoSchema != nil && (tablecodec.IsRecordKey(key) || tablecodec.IsIndexKey(key)) { + tableID := tablecodec.DecodeTableID(key) + if table, ok := infoSchema.TableByID(tableID); ok { + if tablecodec.IsRecordKey(key) { + decodeFunc = consistency.DecodeRowMvccData(table.Meta()) + } else { + tableInfo := table.Meta() + _, indexID, _, e := tablecodec.DecodeIndexKey(key) + if e != nil { + logutil.Logger(ctx).Error("assertion failed but cannot decode index key", zap.Error(e)) + return err + } + var indexInfo *model.IndexInfo + for _, idx := range tableInfo.Indices { + if idx.ID == indexID { + indexInfo = idx + break + } + } + if indexInfo == nil { + return err + } + decodeFunc = consistency.DecodeIndexMvccData(indexInfo) + } + } else { + logutil.Logger(ctx).Warn("assertion failed but table not found in infoschema", zap.Int64("tableID", tableID)) + } + } + if store, ok := s.store.(helper.Storage); ok { + content := consistency.GetMvccByKey(store, key, decodeFunc) + logutil.Logger(ctx).Error("assertion failed", zap.String("message", newErr.Error()), zap.String("mvcc history", content)) + } + return newErr +} + func (s *session) commitTxnWithTemporaryData(ctx context.Context, txn kv.Transaction) error { sessVars := s.sessionVars txnTempTables := sessVars.TxnCtx.TemporaryTables @@ -1142,8 +1207,7 @@ func createSessionFunc(store kv.Storage) pools.Factory { } se.sessionVars.CommonGlobalLoaded = true se.sessionVars.InRestrictedSQL = true - // TODO: Remove this line after fixing https://github.com/pingcap/tidb/issues/30880 - // Chunk RPC protocol may have memory leak issue not solved. + // Internal session uses default format to prevent memory leak problem. se.sessionVars.EnableChunkRPC = false return se, nil } @@ -1165,8 +1229,7 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R } se.sessionVars.CommonGlobalLoaded = true se.sessionVars.InRestrictedSQL = true - // TODO: Remove this line after fixing https://github.com/pingcap/tidb/issues/30880 - // Chunk RPC protocol may have memory leak issue not solved. + // Internal session uses default format to prevent memory leak problem. se.sessionVars.EnableChunkRPC = false return se, nil } @@ -1192,11 +1255,7 @@ func drainRecordSet(ctx context.Context, se *session, rs sqlexec.RecordSet, allo // getTableValue executes restricted sql and the result is one column. // It returns a string value. func (s *session) getTableValue(ctx context.Context, tblName string, varName string) (string, error) { - stmt, err := s.ParseWithParams(ctx, true, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?", mysql.SystemDB, tblName, varName) - if err != nil { - return "", err - } - rows, fields, err := s.ExecRestrictedStmt(ctx, stmt) + rows, fields, err := s.ExecRestrictedSQL(ctx, nil, "SELECT VARIABLE_VALUE FROM %n.%n WHERE VARIABLE_NAME=%?", mysql.SystemDB, tblName, varName) if err != nil { return "", err } @@ -1214,11 +1273,10 @@ func (s *session) getTableValue(ctx context.Context, tblName string, varName str // replaceGlobalVariablesTableValue executes restricted sql updates the variable value // It will then notify the etcd channel that the value has changed. func (s *session) replaceGlobalVariablesTableValue(ctx context.Context, varName, val string) error { - stmt, err := s.ParseWithParams(ctx, true, `REPLACE INTO %n.%n (variable_name, variable_value) VALUES (%?, %?)`, mysql.SystemDB, mysql.GlobalVariablesTable, varName, val) + _, _, err := s.ExecRestrictedSQL(ctx, nil, `REPLACE INTO %n.%n (variable_name, variable_value) VALUES (%?, %?)`, mysql.SystemDB, mysql.GlobalVariablesTable, varName, val) if err != nil { return err } - _, _, err = s.ExecRestrictedStmt(ctx, stmt) domain.GetDomain(s).NotifyUpdateSysVarCache() return err } @@ -1289,11 +1347,7 @@ func (s *session) SetGlobalSysVarOnly(name, value string) (err error) { // SetTiDBTableValue implements GlobalVarAccessor.SetTiDBTableValue interface. func (s *session) SetTiDBTableValue(name, value, comment string) error { - stmt, err := s.ParseWithParams(context.TODO(), true, `REPLACE INTO mysql.tidb (variable_name, variable_value, comment) VALUES (%?, %?, %?)`, name, value, comment) - if err != nil { - return err - } - _, _, err = s.ExecRestrictedStmt(context.TODO(), stmt) + _, _, err := s.ExecRestrictedSQL(context.TODO(), nil, `REPLACE INTO mysql.tidb (variable_name, variable_value, comment) VALUES (%?, %?, %?)`, name, value, comment) return err } @@ -1421,7 +1475,7 @@ func (s *session) ExecuteInternal(ctx context.Context, sql string, args ...inter logutil.Eventf(ctx, "execute: %s", sql) } - stmtNode, err := s.ParseWithParams(ctx, true, sql, args...) + stmtNode, err := s.ParseWithParams(ctx, sql, args...) if err != nil { return nil, err } @@ -1499,7 +1553,7 @@ func (s *session) Parse(ctx context.Context, sql string) ([]ast.StmtNode, error) // ParseWithParams parses a query string, with arguments, to raw ast.StmtNode. // Note that it will not do escaping if no variable arguments are passed. -func (s *session) ParseWithParams(ctx context.Context, forceUTF8SQL bool, sql string, args ...interface{}) (ast.StmtNode, error) { +func (s *session) ParseWithParams(ctx context.Context, sql string, args ...interface{}) (ast.StmtNode, error) { var err error if len(args) > 0 { sql, err = sqlexec.EscapeSQL(sql, args...) @@ -1513,7 +1567,7 @@ func (s *session) ParseWithParams(ctx context.Context, forceUTF8SQL bool, sql st var stmts []ast.StmtNode var warns []error parseStartTime := time.Now() - if internal || forceUTF8SQL { + if internal { // Do no respect the settings from clients, if it is for internal usage. // Charsets from clients may give chance injections. // Refer to https://stackoverflow.com/questions/5741187/sql-injection-that-gets-around-mysql-real-escape-string/12118602. @@ -1563,39 +1617,86 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, if topsqlstate.TopSQLEnabled() { defer pprof.SetGoroutineLabels(ctx) } - var execOption sqlexec.ExecOption - for _, opt := range opts { - opt(&execOption) + execOption := sqlexec.GetExecOption(opts) + var se *session + var clean func() + var err error + if execOption.UseCurSession { + se, clean, err = s.useCurrentSession(execOption) + } else { + se, clean, err = s.getInternalSession(execOption) } - // Use special session to execute the sql. - tmp, err := s.sysSessionPool().Get() if err != nil { return nil, nil, err } - defer s.sysSessionPool().Put(tmp) - se := tmp.(*session) + defer clean() startTime := time.Now() + metrics.SessionRestrictedSQLCounter.Inc() + ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) + ctx = context.WithValue(ctx, tikvutil.ExecDetailsKey, &tikvutil.ExecDetails{}) + rs, err := se.ExecuteStmt(ctx, stmtNode) + if err != nil { + se.sessionVars.StmtCtx.AppendError(err) + } + if rs == nil { + return nil, nil, err + } + defer func() { + if closeErr := rs.Close(); closeErr != nil { + err = closeErr + } + }() + var rows []chunk.Row + rows, err = drainRecordSet(ctx, se, rs, nil) + if err != nil { + return nil, nil, err + } + metrics.QueryDurationHistogram.WithLabelValues(metrics.LblInternal).Observe(time.Since(startTime).Seconds()) + return rows, rs.Fields(), err +} + +// only set and clean session with execOption +func (s *session) useCurrentSession(execOption sqlexec.ExecOption) (*session, func(), error) { + var err error + if execOption.SnapshotTS != 0 { + s.sessionVars.SnapshotInfoschema, err = getSnapshotInfoSchema(s, execOption.SnapshotTS) + if err != nil { + return nil, nil, err + } + if err := s.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { + return nil, nil, err + } + } + prevStatsVer := s.sessionVars.AnalyzeVersion + if execOption.AnalyzeVer != 0 { + s.sessionVars.AnalyzeVersion = execOption.AnalyzeVer + } + return s, func() { + s.sessionVars.AnalyzeVersion = prevStatsVer + if err := s.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { + logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) + } + s.sessionVars.SnapshotInfoschema = nil + }, nil +} + +func (s *session) getInternalSession(execOption sqlexec.ExecOption) (*session, func(), error) { + tmp, err := s.sysSessionPool().Get() + if err != nil { + return nil, nil, errors.Trace(err) + } + se := tmp.(*session) + // The special session will share the `InspectionTableCache` with current session // if the current session in inspection mode. if cache := s.sessionVars.InspectionTableCache; cache != nil { se.sessionVars.InspectionTableCache = cache - defer func() { se.sessionVars.InspectionTableCache = nil }() } if ok := s.sessionVars.OptimizerUseInvisibleIndexes; ok { se.sessionVars.OptimizerUseInvisibleIndexes = true - defer func() { se.sessionVars.OptimizerUseInvisibleIndexes = false }() } prePruneMode := se.sessionVars.PartitionPruneMode.Load() - defer func() { - if !execOption.IgnoreWarning { - if se != nil && se.GetSessionVars().StmtCtx.WarningCount() > 0 { - warnings := se.GetSessionVars().StmtCtx.GetWarnings() - s.GetSessionVars().StmtCtx.AppendWarnings(warnings) - } - } - se.sessionVars.PartitionPruneMode.Store(prePruneMode) - }() if execOption.SnapshotTS != 0 { se.sessionVars.SnapshotInfoschema, err = getSnapshotInfoSchema(s, execOption.SnapshotTS) @@ -1605,47 +1706,85 @@ func (s *session) ExecRestrictedStmt(ctx context.Context, stmtNode ast.StmtNode, if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, strconv.FormatUint(execOption.SnapshotTS, 10)); err != nil { return nil, nil, err } - defer func() { - if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { - logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) - } - se.sessionVars.SnapshotInfoschema = nil - }() } + prevStatsVer := se.sessionVars.AnalyzeVersion if execOption.AnalyzeVer != 0 { - prevStatsVer := se.sessionVars.AnalyzeVersion se.sessionVars.AnalyzeVersion = execOption.AnalyzeVer - defer func() { - se.sessionVars.AnalyzeVersion = prevStatsVer - }() } // for analyze stmt we need let worker session follow user session that executing stmt. se.sessionVars.PartitionPruneMode.Store(s.sessionVars.PartitionPruneMode.Load()) - metrics.SessionRestrictedSQLCounter.Inc() - ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) - ctx = context.WithValue(ctx, tikvutil.ExecDetailsKey, &tikvutil.ExecDetails{}) - rs, err := se.ExecuteStmt(ctx, stmtNode) - if err != nil { - se.sessionVars.StmtCtx.AppendError(err) - } - if rs == nil { - return nil, nil, err - } - defer func() { - if closeErr := rs.Close(); closeErr != nil { - err = closeErr + return se, func() { + se.sessionVars.AnalyzeVersion = prevStatsVer + if err := se.sessionVars.SetSystemVar(variable.TiDBSnapshot, ""); err != nil { + logutil.BgLogger().Error("set tidbSnapshot error", zap.Error(err)) } - }() - var rows []chunk.Row - rows, err = drainRecordSet(ctx, se, rs, nil) + se.sessionVars.SnapshotInfoschema = nil + if !execOption.IgnoreWarning { + if se != nil && se.GetSessionVars().StmtCtx.WarningCount() > 0 { + warnings := se.GetSessionVars().StmtCtx.GetWarnings() + s.GetSessionVars().StmtCtx.AppendWarnings(warnings) + } + } + se.sessionVars.PartitionPruneMode.Store(prePruneMode) + se.sessionVars.OptimizerUseInvisibleIndexes = false + se.sessionVars.InspectionTableCache = nil + s.sysSessionPool().Put(tmp) + }, nil +} + +func (s *session) withRestrictedSQLExecutor(ctx context.Context, opts []sqlexec.OptionFuncAlias, fn func(context.Context, *session) ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { + execOption := sqlexec.GetExecOption(opts) + var se *session + var clean func() + var err error + if execOption.UseCurSession { + se, clean, err = s.useCurrentSession(execOption) + } else { + se, clean, err = s.getInternalSession(execOption) + } if err != nil { - return nil, nil, err + return nil, nil, errors.Trace(err) } - metrics.QueryDurationHistogram.WithLabelValues(metrics.LblInternal).Observe(time.Since(startTime).Seconds()) - return rows, rs.Fields(), err + defer clean() + return fn(ctx, se) +} + +func (s *session) ExecRestrictedSQL(ctx context.Context, opts []sqlexec.OptionFuncAlias, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { + return s.withRestrictedSQLExecutor(ctx, opts, func(ctx context.Context, se *session) ([]chunk.Row, []*ast.ResultField, error) { + stmt, err := se.ParseWithParams(ctx, sql, params...) + if err != nil { + return nil, nil, errors.Trace(err) + } + if topsqlstate.TopSQLEnabled() { + defer pprof.SetGoroutineLabels(ctx) + } + startTime := time.Now() + metrics.SessionRestrictedSQLCounter.Inc() + ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) + ctx = context.WithValue(ctx, tikvutil.ExecDetailsKey, &tikvutil.ExecDetails{}) + rs, err := se.ExecuteStmt(ctx, stmt) + if err != nil { + se.sessionVars.StmtCtx.AppendError(err) + } + if rs == nil { + return nil, nil, err + } + defer func() { + if closeErr := rs.Close(); closeErr != nil { + err = closeErr + } + }() + var rows []chunk.Row + rows, err = drainRecordSet(ctx, se, rs, nil) + if err != nil { + return nil, nil, err + } + metrics.QueryDurationHistogram.WithLabelValues(metrics.LblInternal).Observe(time.Since(startTime).Seconds()) + return rows, rs.Fields(), err + }) } func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlexec.RecordSet, error) { @@ -2177,6 +2316,19 @@ func (s *session) DropPreparedStmt(stmtID uint32) error { return nil } +// setTxnAssertionLevel sets assertion level of a transactin. Note that assertion level should be set only once just +// after creating a new transaction. +func setTxnAssertionLevel(txn kv.Transaction, assertionLevel variable.AssertionLevel) { + switch assertionLevel { + case variable.AssertionLevelOff: + txn.SetOption(kv.AssertionLevel, kvrpcpb.AssertionLevel_Off) + case variable.AssertionLevelFast: + txn.SetOption(kv.AssertionLevel, kvrpcpb.AssertionLevel_Fast) + case variable.AssertionLevelStrict: + txn.SetOption(kv.AssertionLevel, kvrpcpb.AssertionLevel_Strict) + } +} + func (s *session) Txn(active bool) (kv.Transaction, error) { if !active { return &s.txn, nil @@ -2215,6 +2367,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { if s.GetSessionVars().StmtCtx.WeakConsistency { s.txn.SetOption(kv.IsolationLevel, kv.RC) } + setTxnAssertionLevel(&s.txn, s.sessionVars.AssertionLevel) } return &s.txn, nil } @@ -2269,6 +2422,7 @@ func (s *session) NewTxn(ctx context.Context) error { if replicaReadType.IsFollowerRead() { txn.SetOption(kv.ReplicaRead, replicaReadType) } + setTxnAssertionLevel(txn, s.sessionVars.AssertionLevel) s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ @@ -2312,6 +2466,7 @@ func (s *session) NewStaleTxnWithStartTS(ctx context.Context, startTS uint64) er txn.SetVars(s.sessionVars.KVVars) txn.SetOption(kv.IsStalenessReadOnly, true) txn.SetOption(kv.TxnScope, txnScope) + setTxnAssertionLevel(txn, s.sessionVars.AssertionLevel) s.txn.changeInvalidToValid(txn) is, err := getSnapshotInfoSchema(s, txn.StartTS()) if err != nil { @@ -2615,9 +2770,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } - if newCollationEnabled { - collate.EnableNewCollations() - } + collate.SetNewCollationEnabledForTest(newCollationEnabled) newMemoryQuotaQuery, err := loadDefMemQuotaQuery(se) if err != nil { @@ -2930,7 +3083,8 @@ func (s *session) PrepareTxnCtx(ctx context.Context) error { ShardStep: int(s.sessionVars.ShardAllocateStep), TxnScope: s.GetSessionVars().CheckAndGetTxnScope(), } - if !s.sessionVars.IsAutocommit() || s.sessionVars.RetryInfo.Retrying { + if !s.sessionVars.IsAutocommit() || s.sessionVars.RetryInfo.Retrying || + config.GetGlobalConfig().PessimisticTxn.PessimisticAutoCommit.Load() { if s.sessionVars.TxnMode == ast.Pessimistic { s.sessionVars.TxnCtx.IsPessimistic = true } @@ -2996,6 +3150,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { return err } txn.SetVars(s.sessionVars.KVVars) + setTxnAssertionLevel(txn, s.sessionVars.AssertionLevel) s.txn.changeInvalidToValid(txn) err = s.loadCommonGlobalVariablesIfNeeded() if err != nil { diff --git a/session/session_fail_test.go b/session/session_fail_test.go index 2e293f106ff6d..2e0b770c348ea 100644 --- a/session/session_fail_test.go +++ b/session/session_fail_test.go @@ -16,16 +16,20 @@ package session_test import ( "context" - "strings" + "testing" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -func (s *testSessionSerialSuite) TestFailStatementCommitInRetry(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestFailStatementCommitInRetry(t *testing.T) { + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) tk.MustExec("create table t (id int)") tk.MustExec("begin") @@ -33,90 +37,109 @@ func (s *testSessionSerialSuite) TestFailStatementCommitInRetry(c *C) { tk.MustExec("insert into t values (2),(3),(4),(5)") tk.MustExec("insert into t values (6)") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError8942", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError8942", `return(true)`)) _, err := tk.Exec("commit") - c.Assert(err, NotNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError8942"), IsNil) + require.Error(t, err) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError8942")) tk.MustExec("insert into t values (6)") tk.MustQuery(`select * from t`).Check(testkit.Rows("6")) } -func (s *testSessionSerialSuite) TestGetTSFailDirtyState(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestGetTSFailDirtyState(t *testing.T) { + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) tk.MustExec("create table t (id int)") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockGetTSFail", "return"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/mockGetTSFail", "return")) ctx := failpoint.WithHook(context.Background(), func(ctx context.Context, fpname string) bool { return fpname == "github.com/pingcap/tidb/session/mockGetTSFail" }) - _, err := tk.Se.Execute(ctx, "select * from t") - c.Assert(err, NotNil) + _, err := tk.Session().Execute(ctx, "select * from t") + if config.GetGlobalConfig().Store == "unistore" { + require.Error(t, err) + } else { + require.NoError(t, err) + } // Fix a bug that active txn fail set TxnState.fail to error, and then the following write // affected by this fail flag. tk.MustExec("insert into t values (1)") tk.MustQuery(`select * from t`).Check(testkit.Rows("1")) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockGetTSFail"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/mockGetTSFail")) } -func (s *testSessionSerialSuite) TestGetTSFailDirtyStateInretry(c *C) { +func TestGetTSFailDirtyStateInretry(t *testing.T) { defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError"), IsNil) - c.Assert(failpoint.Disable("tikvclient/mockGetTSErrorInRetry"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError")) + require.NoError(t, failpoint.Disable("tikvclient/mockGetTSErrorInRetry")) }() - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) tk.MustExec("create table t (id int)") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError", `return(true)`)) // This test will mock a PD timeout error, and recover then. // Just make mockGetTSErrorInRetry return true once, and then return false. - c.Assert(failpoint.Enable("tikvclient/mockGetTSErrorInRetry", - `1*return(true)->return(false)`), IsNil) + require.NoError(t, failpoint.Enable("tikvclient/mockGetTSErrorInRetry", + `1*return(true)->return(false)`)) tk.MustExec("insert into t values (2)") tk.MustQuery(`select * from t`).Check(testkit.Rows("2")) } -func (s *testSessionSerialSuite) TestKillFlagInBackoff(c *C) { +func TestKillFlagInBackoff(t *testing.T) { // This test checks the `killed` flag is passed down to the backoffer through // session.KVVars. - tk := testkit.NewTestKitWithInit(c, s.store) + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) tk.MustExec("create table kill_backoff (id int)") // Inject 1 time timeout. If `Killed` is not successfully passed, it will retry and complete query. - c.Assert(failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("timeout")->return("")`), IsNil) + require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("timeout")->return("")`)) defer failpoint.Disable("tikvclient/tikvStoreSendReqResult") // Set kill flag and check its passed to backoffer. - tk.Se.GetSessionVars().Killed = 1 + tk.Session().GetSessionVars().Killed = 1 rs, err := tk.Exec("select * from kill_backoff") - c.Assert(err, IsNil) - _, err = session.ResultSetToStringSlice(context.TODO(), tk.Se, rs) + require.NoError(t, err) + _, err = session.ResultSetToStringSlice(context.TODO(), tk.Session(), rs) // `interrupted` is returned when `Killed` is set. - c.Assert(strings.Contains(err.Error(), "Query execution was interrupted"), IsTrue) + require.Regexp(t, ".*Query execution was interrupted.*", err.Error()) } -func (s *testSessionSerialSuite) TestClusterTableSendError(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - c.Assert(failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("requestTiDBStoreError")`), IsNil) +func TestClusterTableSendError(t *testing.T) { + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) + require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("requestTiDBStoreError")`)) defer failpoint.Disable("tikvclient/tikvStoreSendReqResult") tk.MustQuery("select * from information_schema.cluster_slow_query") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err, ErrorMatches, ".*TiDB server timeout, address is.*") + require.Equal(t, tk.Session().GetSessionVars().StmtCtx.WarningCount(), uint16(1)) + require.Regexp(t, ".*TiDB server timeout, address is.*", tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error()) } -func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) +func TestAutoCommitNeedNotLinearizability(t *testing.T) { + store, clean := createStorage(t) + defer clean() + + tk := createTestKit(t, store) tk.MustExec("drop table if exists t1;") defer tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1 (c int)`) - c.Assert(failpoint.Enable("tikvclient/getMinCommitTSFromTSO", `panic`), IsNil) + require.NoError(t, failpoint.Enable("tikvclient/getMinCommitTSFromTSO", `panic`)) defer func() { - c.Assert(failpoint.Disable("tikvclient/getMinCommitTSFromTSO"), IsNil) + require.NoError(t, failpoint.Disable("tikvclient/getMinCommitTSFromTSO")) }() - c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_enable_async_commit", "1"), IsNil) - c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_guarantee_linearizability", "1"), IsNil) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tidb_enable_async_commit", "1")) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tidb_guarantee_linearizability", "1")) // Auto-commit transactions don't need to get minCommitTS from TSO tk.MustExec("INSERT INTO t1 VALUES (1)") @@ -127,7 +150,7 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) { func() { defer func() { err := recover() - c.Assert(err, NotNil) + require.NotNil(t, err) }() tk.MustExec("COMMIT") }() @@ -137,14 +160,14 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) { func() { defer func() { err := recover() - c.Assert(err, NotNil) + require.NotNil(t, err) }() tk.MustExec("COMMIT") }() // Same for 1PC tk.MustExec("set autocommit = 1") - c.Assert(tk.Se.GetSessionVars().SetSystemVar("tidb_enable_1pc", "1"), IsNil) + require.NoError(t, tk.Session().GetSessionVars().SetSystemVar("tidb_enable_1pc", "1")) tk.MustExec("INSERT INTO t1 VALUES (4)") tk.MustExec("BEGIN") @@ -152,7 +175,7 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) { func() { defer func() { err := recover() - c.Assert(err, NotNil) + require.NotNil(t, err) }() tk.MustExec("COMMIT") }() @@ -162,7 +185,7 @@ func (s *testSessionSerialSuite) TestAutoCommitNeedNotLinearizability(c *C) { func() { defer func() { err := recover() - c.Assert(err, NotNil) + require.NotNil(t, err) }() tk.MustExec("COMMIT") }() diff --git a/session/session_test.go b/session/session_test.go index ac68f7fab7d2c..31a932d052c73 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -27,6 +27,7 @@ import ( "strings" "sync" "sync/atomic" + "testing" "time" "github.com/docker/go-units" @@ -61,18 +62,19 @@ import ( "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + newTestkit "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tipb/go-binlog" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/txnkv/transaction" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "google.golang.org/grpc" ) @@ -259,6 +261,33 @@ func (s *testSessionSuiteBase) TearDownTest(c *C) { } } +func createStorage(t *testing.T) (store kv.Storage, clean func()) { + if *withTiKV { + initPdAddrs() + pdAddr := <-pdAddrChan + var d driver.TiKVDriver + config.UpdateGlobal(func(conf *config.Config) { + conf.TxnLocalLatches.Enabled = false + }) + store, err := d.Open(fmt.Sprintf("tikv://%s?disableGC=true", pdAddr)) + require.NoError(t, err) + err = clearStorage(store) + require.NoError(t, err) + err = clearETCD(store.(kv.EtcdBackend)) + require.NoError(t, err) + session.ResetStoreForWithTiKVTest(store) + dom, err := session.BootstrapSession(store) + require.NoError(t, err) + + return store, func() { + dom.Close() + store.Close() + pdAddrChan <- pdAddr + } + } + return newTestkit.CreateMockStore(t) +} + type mockBinlogPump struct { } @@ -4420,11 +4449,11 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { exec := se.(sqlexec.RestrictedSQLExecutor) // test compatibility with ExcuteInternal - _, err := exec.ParseWithParams(context.TODO(), true, "SELECT 4") + _, err := exec.ParseWithParams(context.TODO(), "SELECT 4") c.Assert(err, IsNil) // test charset attack - stmt, err := exec.ParseWithParams(context.TODO(), true, "SELECT * FROM test WHERE name = %? LIMIT 1", "\xbf\x27 OR 1=1 /*") + stmt, err := exec.ParseWithParams(context.TODO(), "SELECT * FROM test WHERE name = %? LIMIT 1", "\xbf\x27 OR 1=1 /*") c.Assert(err, IsNil) var sb strings.Builder @@ -4434,15 +4463,15 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { c.Assert(sb.String(), Equals, "SELECT * FROM test WHERE name=_utf8mb4\"\xbf' OR 1=1 /*\" LIMIT 1") // test invalid sql - _, err = exec.ParseWithParams(context.TODO(), true, "SELECT") + _, err = exec.ParseWithParams(context.TODO(), "SELECT") c.Assert(err, ErrorMatches, ".*You have an error in your SQL syntax.*") // test invalid arguments to escape - _, err = exec.ParseWithParams(context.TODO(), true, "SELECT %?, %?", 3) + _, err = exec.ParseWithParams(context.TODO(), "SELECT %?, %?", 3) c.Assert(err, ErrorMatches, "missing arguments.*") // test noescape - stmt, err = exec.ParseWithParams(context.TODO(), true, "SELECT 3") + stmt, err = exec.ParseWithParams(context.TODO(), "SELECT 3") c.Assert(err, IsNil) sb.Reset() @@ -4905,8 +4934,6 @@ func (s *testStatisticsSuite) cleanEnv(c *C, store kv.Storage, do *domain.Domain } func (s *testStatisticsSuite) TestNewCollationStatsWithPrefixIndex(c *C) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) defer s.cleanEnv(c, s.store, s.dom) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -5927,13 +5954,16 @@ func (s *testSessionSuite) TestWriteOnMultipleCachedTable(c *C) { tk.MustQuery("select * from ct1").Check(testkit.Rows()) tk.MustQuery("select * from ct2").Check(testkit.Rows()) + lastReadFromCache := func(tk *testkit.TestKit) bool { + return tk.Se.GetSessionVars().StmtCtx.ReadFromTableCache + } + cached := false for i := 0; i < 50; i++ { - if tk.HasPlan("select * from ct1", "Union") { - if tk.HasPlan("select * from ct2", "Union") { - cached = true - break - } + tk.MustQuery("select * from ct1") + if lastReadFromCache(tk) { + cached = true + break } time.Sleep(100 * time.Millisecond) } @@ -5975,3 +6005,13 @@ func (s *testSessionSuite) TestForbidSettingBothTSVariable(c *C) { tk.MustExec("set @@tidb_read_staleness = ''") tk.MustExec("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'") } + +func (s *testSessionSuite) TestSysdateIsNow(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustQuery("show variables like '%sysdate_is_now%'").Check(testkit.Rows("sysdate_is_now OFF")) + c.Assert(tk.Se.GetSessionVars().SysdateIsNow, IsFalse) + tk.MustExec("set @@sysdate_is_now=true") + tk.MustQuery("show variables like '%sysdate_is_now%'").Check(testkit.Rows("sysdate_is_now ON")) + c.Assert(tk.Se.GetSessionVars().SysdateIsNow, IsTrue) +} diff --git a/session/tidb_test.go b/session/tidb_test.go index 47dc3896ba90a..70831a8f64d89 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -37,7 +37,7 @@ func TestSysSessionPoolGoroutineLeak(t *testing.T) { count := 200 stmts := make([]ast.StmtNode, count) for i := 0; i < count; i++ { - stmt, err := se.ParseWithParams(context.Background(), true, "select * from mysql.user limit 1") + stmt, err := se.ParseWithParams(context.Background(), "select * from mysql.user limit 1") require.NoError(t, err) stmts[i] = stmt } diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 0eee0d77eafdc..f4ad7216cd7a5 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -30,13 +30,11 @@ import ( "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" @@ -112,6 +110,7 @@ func createBinlogSuite(t *testing.T) (s *binlogSuite, clean func()) { s.ddl.SetBinlogClient(s.client) clean = func() { + clientCon.Close() err = s.ddl.Stop() require.NoError(t, err) s.serv.Stop() @@ -404,7 +403,7 @@ func TestBinlogForSequence(t *testing.T) { tk.MustExec("create sequence seq cache 3") // trigger the sequence cache allocation. tk.MustQuery("select nextval(seq)").Check(testkit.Rows("1")) - sequenceTable := testGetTableByName(t, tk.Session(), "test", "seq") + sequenceTable := testkit.TestGetTableByName(t, tk.Session(), "test", "seq") tc, ok := sequenceTable.(*tables.TableCommon) require.Equal(t, true, ok) _, end, round := tc.GetSequenceCommon().GetSequenceBaseEndRound() @@ -432,7 +431,7 @@ func TestBinlogForSequence(t *testing.T) { tk.MustExec("create sequence seq2 start 1 increment -2 cache 3 minvalue -10 maxvalue 10 cycle") // trigger the sequence cache allocation. tk.MustQuery("select nextval(seq2)").Check(testkit.Rows("1")) - sequenceTable = testGetTableByName(t, tk.Session(), "test2", "seq2") + sequenceTable = testkit.TestGetTableByName(t, tk.Session(), "test2", "seq2") tc, ok = sequenceTable.(*tables.TableCommon) require.Equal(t, true, ok) _, end, round = tc.GetSequenceCommon().GetSequenceBaseEndRound() @@ -725,16 +724,6 @@ func mustGetDDLBinlog(s *binlogSuite, ddlQuery string, t *testing.T) (matched bo return } -func testGetTableByName(t *testing.T, ctx sessionctx.Context, db, table string) table.Table { - dom := domain.GetDomain(ctx) - // Make sure the table schema is the new schema. - err := dom.Reload() - require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) - require.NoError(t, err) - return tbl -} - func TestTempTableBinlog(t *testing.T) { s, clean := createBinlogSuite(t) defer clean() diff --git a/sessionctx/binloginfo/main_test.go b/sessionctx/binloginfo/main_test.go index c7e1ff6d026be..4462e494a041a 100644 --- a/sessionctx/binloginfo/main_test.go +++ b/sessionctx/binloginfo/main_test.go @@ -24,10 +24,8 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("google.golang.org/grpc.(*addrConn).resetTransport"), - goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), } goleak.VerifyTestMain(m, opts...) } diff --git a/sessionctx/stmtctx/main_test.go b/sessionctx/stmtctx/main_test.go index 4e2b518ab545c..ced7d8000ece8 100644 --- a/sessionctx/stmtctx/main_test.go +++ b/sessionctx/stmtctx/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index dd63a94e8b7f6..ce8f10c875f8f 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -235,6 +235,9 @@ type StatementContext struct { // LoadStartTime is to record the load start time to calculate latency LoadStartTime time.Time } + + // SysdateIsNow indicates whether sysdate() is an alias of now() in this statement + SysdateIsNow bool } // StmtHints are SessionVars related sql hints. diff --git a/sessionctx/stmtctx/stmtctx_test.go b/sessionctx/stmtctx/stmtctx_test.go index 7b27e2c62a653..7a4ec77a90660 100644 --- a/sessionctx/stmtctx/stmtctx_test.go +++ b/sessionctx/stmtctx/stmtctx_test.go @@ -15,10 +15,12 @@ package stmtctx_test import ( + "context" "fmt" "testing" "time" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/execdetails" @@ -96,49 +98,48 @@ func TestWeakConsistencyRead(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() - lastWeakConsistency := func(tk *testkit.TestKit) bool { - return tk.Session().GetSessionVars().StmtCtx.WeakConsistency - } - tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(id int primary key, c int, c1 int, unique index i(c))") + + execAndCheck := func(sql string, rows [][]interface{}, isolationLevel kv.IsoLevel) { + ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { + require.Equal(t, req.IsolationLevel, isolationLevel) + }) + rss, err := tk.Session().Execute(ctx, sql) + require.Nil(t, err) + for _, rs := range rss { + rs.Close() + } + if rows != nil { + tk.MustQuery(sql).Check(rows) + } + lastWeakConsistency := tk.Session().GetSessionVars().StmtCtx.WeakConsistency + require.Equal(t, lastWeakConsistency, isolationLevel == kv.RC) + } + // strict - tk.MustExec("insert into t values(1, 1, 1)") - require.False(t, lastWeakConsistency(tk)) - tk.MustQuery("select * from t").Check(testkit.Rows("1 1 1")) - require.False(t, lastWeakConsistency(tk)) + execAndCheck("insert into t values(1, 1, 1)", nil, kv.SI) + execAndCheck("select * from t", testkit.Rows("1 1 1"), kv.SI) tk.MustExec("prepare s from 'select * from t'") tk.MustExec("prepare u from 'update t set c1 = id + 1'") - tk.MustQuery("execute s").Check(testkit.Rows("1 1 1")) - require.False(t, lastWeakConsistency(tk)) - tk.MustExec("execute u") - require.False(t, lastWeakConsistency(tk)) - tk.MustExec("admin check table t") - require.False(t, lastWeakConsistency(tk)) + execAndCheck("execute s", testkit.Rows("1 1 1"), kv.SI) + execAndCheck("execute u", nil, kv.SI) + execAndCheck("admin check table t", nil, kv.SI) // weak tk.MustExec("set tidb_read_consistency = weak") - tk.MustExec("insert into t values(2, 2, 2)") - require.False(t, lastWeakConsistency(tk)) - tk.MustQuery("select * from t").Check(testkit.Rows("1 1 2", "2 2 2")) - require.True(t, lastWeakConsistency(tk)) - tk.MustQuery("execute s").Check(testkit.Rows("1 1 2", "2 2 2")) - require.True(t, lastWeakConsistency(tk)) - tk.MustExec("execute u") - require.False(t, lastWeakConsistency(tk)) + execAndCheck("insert into t values(2, 2, 2)", nil, kv.SI) + execAndCheck("select * from t", testkit.Rows("1 1 2", "2 2 2"), kv.RC) + execAndCheck("execute s", testkit.Rows("1 1 2", "2 2 2"), kv.RC) + execAndCheck("execute u", nil, kv.SI) // non-read-only queries should be strict - tk.MustExec("admin check table t") - require.False(t, lastWeakConsistency(tk)) - tk.MustExec("update t set c = c + 1 where id = 2") - require.False(t, lastWeakConsistency(tk)) - tk.MustExec("delete from t where id = 2") - require.False(t, lastWeakConsistency(tk)) + execAndCheck("admin check table t", nil, kv.SI) + execAndCheck("update t set c = c + 1 where id = 2", nil, kv.SI) + execAndCheck("delete from t where id = 2", nil, kv.SI) // in-transaction queries should be strict tk.MustExec("begin") - tk.MustQuery("select * from t").Check(testkit.Rows("1 1 2")) - require.False(t, lastWeakConsistency(tk)) - tk.MustQuery("execute s").Check(testkit.Rows("1 1 2")) - require.False(t, lastWeakConsistency(tk)) + execAndCheck("select * from t", testkit.Rows("1 1 2"), kv.SI) + execAndCheck("execute s", testkit.Rows("1 1 2"), kv.SI) tk.MustExec("rollback") } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index b0e5e64ba27fe..f1111bbdbe433 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -918,6 +918,9 @@ type SessionVars struct { // LastQueryInfo keeps track the info of last query. LastQueryInfo QueryInfo + // LastDDLInfo keeps track the info of last DDL. + LastDDLInfo LastDDLInfo + // PartitionPruneMode indicates how and when to prune partitions. PartitionPruneMode atomic2.String @@ -1001,6 +1004,13 @@ type SessionVars struct { // StatsLoadSyncWait indicates how long to wait for stats load before timeout. StatsLoadSyncWait int64 + + // SysdateIsNow indicates whether Sysdate is an alias of Now function + SysdateIsNow bool + // EnableMutationChecker indicates whether to check data consistency for mutations + EnableMutationChecker bool + // AssertionLevel controls how strict the assertions on data mutations should be. + AssertionLevel AssertionLevel } // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. @@ -1648,20 +1658,6 @@ func (s *SessionVars) GetReadableTxnMode() string { return txnMode } -func (s *SessionVars) setTxnMode(val string) error { - switch strings.ToUpper(val) { - case ast.Pessimistic: - s.TxnMode = ast.Pessimistic - case ast.Optimistic: - s.TxnMode = ast.Optimistic - case "": - s.TxnMode = "" - default: - return ErrWrongValueForVar.FastGenByArgs(TiDBTxnMode, val) - } - return nil -} - // SetPrevStmtDigest sets the digest of the previous statement. func (s *SessionVars) SetPrevStmtDigest(prevStmtDigest string) { s.prevStmtDigest = prevStmtDigest @@ -1676,7 +1672,7 @@ func (s *SessionVars) GetPrevStmtDigest() string { // LazyCheckKeyNotExists returns if we can lazy check key not exists. func (s *SessionVars) LazyCheckKeyNotExists() bool { - return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) + return s.PresumeKeyNotExists || (s.TxnCtx != nil && s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) } // GetTemporaryTable returns a TempTable by tableInfo. @@ -2163,7 +2159,7 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { } if len(s.CurrentDB) > 0 { - writeSlowLogItem(&buf, SlowLogDBStr, s.CurrentDB) + writeSlowLogItem(&buf, SlowLogDBStr, strings.ToLower(s.CurrentDB)) } if len(logItems.IndexNames) > 0 { writeSlowLogItem(&buf, SlowLogIndexNamesStr, logItems.IndexNames) @@ -2274,7 +2270,7 @@ func (s *SessionVars) SlowLogFormat(logItems *SlowQueryLogItems) string { } if s.CurrentDBChanged { - buf.WriteString(fmt.Sprintf("use %s;\n", s.CurrentDB)) + buf.WriteString(fmt.Sprintf("use %s;\n", strings.ToLower(s.CurrentDB))) s.CurrentDBChanged = false } @@ -2298,6 +2294,12 @@ type QueryInfo struct { ErrMsg string `json:"error,omitempty"` } +// LastDDLInfo represents the information of last DDL. It's used to expose information for test purpose. +type LastDDLInfo struct { + Query string `json:"query"` + SeqNum uint64 `json:"seq_num"` +} + // TxnReadTS indicates the value and used situation for tx_read_ts type TxnReadTS struct { readTS uint64 diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index c3b376934ce74..1c722570897df 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -152,7 +152,8 @@ func TestSlowLogFormat(t *testing.T) { seVar.User = &auth.UserIdentity{Username: "root", Hostname: "192.168.0.1"} seVar.ConnectionInfo = &variable.ConnectionInfo{ClientIP: "192.168.0.1"} seVar.ConnectionID = 1 - seVar.CurrentDB = "test" + // the out put of the loged CurrentDB should be 'test', should be to lower cased. + seVar.CurrentDB = "TeST" seVar.InRestrictedSQL = true seVar.StmtCtx.WaitLockLeaseTime = 1 txnTS := uint64(406649736972468225) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 2d3343d0191e2..6476d3cbea9e1 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1375,6 +1375,37 @@ var defaultSysVars = []*SysVar{ return nil }, }, + {Scope: ScopeGlobal | ScopeSession, Name: SysdateIsNow, Value: BoolToOnOff(DefSysdateIsNow), skipInit: true, Type: TypeBool, + SetSession: func(vars *SessionVars, s string) error { + vars.SysdateIsNow = TiDBOptOn(s) + return nil + }, + GetGlobal: func(vars *SessionVars) (s string, err error) { + return strconv.FormatBool(GlobalSysdateIsNow.Load()), nil + }, + SetGlobal: func(vars *SessionVars, s string) error { + GlobalSysdateIsNow.Store(TiDBOptOn(s)) + return nil + }, + }, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableMutationChecker, Hidden: true, + Value: BoolToOnOff(DefTiDBEnableMutationChecker), Type: TypeBool, + SetSession: func(s *SessionVars, val string) error { + s.EnableMutationChecker = TiDBOptOn(val) + return nil + }, + }, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTxnAssertionLevel, Value: DefTiDBTxnAssertionLevel, PossibleValues: []string{AssertionOffStr, AssertionFastStr, AssertionStrictStr}, Hidden: true, Type: TypeEnum, SetSession: func(s *SessionVars, val string) error { + s.AssertionLevel = tidbOptAssertionLevel(val) + return nil + }}, + {Scope: ScopeSession, Name: TiDBLastDDLInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + info, err := json.Marshal(s.LastDDLInfo) + if err != nil { + return "", err + } + return string(info), nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. @@ -1684,4 +1715,6 @@ const ( RandSeed1 = "rand_seed1" // RandSeed2 is the name of 'rand_seed2' system variable. RandSeed2 = "rand_seed2" + // SysdateIsNow is the name of the `sysdate_is_now` system variable + SysdateIsNow = "sysdate_is_now" ) diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 52241bd563a4e..ba90d3e0daa9a 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -60,20 +60,6 @@ func TestSysVar(t *testing.T) { require.Equal(t, runtime.GOARCH, f.Value) } -func TestTxnMode(t *testing.T) { - seVar := NewSessionVars() - require.NotNil(t, seVar) - require.Equal(t, "", seVar.TxnMode) - err := seVar.setTxnMode("pessimistic") - require.NoError(t, err) - err = seVar.setTxnMode("optimistic") - require.NoError(t, err) - err = seVar.setTxnMode("") - require.NoError(t, err) - err = seVar.setTxnMode("something else") - require.Error(t, err) -} - func TestError(t *testing.T) { kvErrs := []*terror.Error{ ErrUnsupportedValueForVar, @@ -99,7 +85,7 @@ func TestRegistrationOfNewSysVar(t *testing.T) { }} RegisterSysVar(&sv) - require.Equal(t, len(GetSysVars()), count+1) + require.Len(t, GetSysVars(), count+1) sysVar := GetSysVar("mynewsysvar") require.NotNil(t, sysVar) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 65bc914a60152..d029d26be1327 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -82,9 +82,12 @@ const ( // TiDBLastTxnInfo is used to get the last transaction info within the current session. TiDBLastTxnInfo = "tidb_last_txn_info" - // TiDBLastTxnInfo is used to get the last query info within the current session. + // TiDBLastQueryInfo is used to get the last query info within the current session. TiDBLastQueryInfo = "tidb_last_query_info" + // TiDBLastDDLInfo is used to get the last ddl info within the current session. + TiDBLastDDLInfo = "tidb_last_ddl_info" + // tidb_config is a read-only variable that shows the config of the current server. TiDBConfig = "tidb_config" @@ -614,6 +617,12 @@ const ( // TiDBStatsLoadSyncWait indicates the time sql execution will sync-wait for stats load. TiDBStatsLoadSyncWait = "tidb_stats_load_sync_wait" + + // TiDBEnableMutationChecker indicates whether to check data consistency for mutations + TiDBEnableMutationChecker = "tidb_enable_mutation_checker" + // TiDBTxnAssertionLevel indicates how strict the assertion will be, which helps detecting and preventing data & + // index inconsistency problems. + TiDBTxnAssertionLevel = "tidb_txn_assertion_level" ) // TiDB vars that have only global scope @@ -809,6 +818,9 @@ const ( DefTiDBEnableColumnTracking = false DefTiDBStatsLoadSyncWait = 0 DefTiDBStatsLoadPseudoTimeout = false + DefSysdateIsNow = false + DefTiDBEnableMutationChecker = false + DefTiDBTxnAssertionLevel = AssertionOffStr ) // Process global variables. @@ -842,4 +854,5 @@ var ( EnableColumnTracking = atomic.NewBool(DefTiDBEnableColumnTracking) StatsLoadSyncWait = atomic.NewInt64(DefTiDBStatsLoadSyncWait) StatsLoadPseudoTimeout = atomic.NewBool(DefTiDBStatsLoadPseudoTimeout) + GlobalSysdateIsNow = atomic.NewBool(DefSysdateIsNow) ) diff --git a/sessionctx/variable/variable.go b/sessionctx/variable/variable.go index a17c3f63e1538..f6a3ae5b47db7 100644 --- a/sessionctx/variable/variable.go +++ b/sessionctx/variable/variable.go @@ -64,6 +64,15 @@ const ( Warn = "WARN" // IntOnly means enable for int type IntOnly = "INT_ONLY" + + // AssertionStrictStr is a choice of variable TiDBTxnAssertionLevel that means full assertions should be performed, + // even if the performance might be slowed down. + AssertionStrictStr = "STRICT" + // AssertionFastStr is a choice of variable TiDBTxnAssertionLevel that means assertions that doesn't affect + // performance should be performed. + AssertionFastStr = "FAST" + // AssertionOffStr is a choice of variable TiDBTxnAssertionLevel that means no assertion should be performed. + AssertionOffStr = "OFF" ) // Global config name list. @@ -554,12 +563,12 @@ func SetSysVar(name string, value string) { func GetSysVars() map[string]*SysVar { sysVarsLock.RLock() defer sysVarsLock.RUnlock() - copy := make(map[string]*SysVar, len(sysVars)) + m := make(map[string]*SysVar, len(sysVars)) for name, sv := range sysVars { tmp := *sv - copy[name] = &tmp + m[name] = &tmp } - return copy + return m } func init() { diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index c82f3788bed97..81ca3502e54d5 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -327,6 +327,31 @@ func TiDBOptEnableClustered(opt string) ClusteredIndexDefMode { } } +// AssertionLevel controls the assertion that will be performed during transactions. +type AssertionLevel int + +const ( + // AssertionLevelOff indicates no assertion should be performed. + AssertionLevelOff AssertionLevel = iota + // AssertionLevelFast indicates assertions that doesn't affect performance should be performed. + AssertionLevelFast + // AssertionLevelStrict indicates full assertions should be performed, even if the performance might be slowed down. + AssertionLevelStrict +) + +func tidbOptAssertionLevel(opt string) AssertionLevel { + switch opt { + case AssertionStrictStr: + return AssertionLevelStrict + case AssertionFastStr: + return AssertionLevelFast + case AssertionOffStr: + return AssertionLevelOff + default: + return AssertionLevelOff + } +} + func tidbOptPositiveInt32(opt string, defaultVal int) int { val, err := strconv.Atoi(opt) if err != nil || val <= 0 { @@ -480,4 +505,5 @@ var GAFunction4ExpressionIndex = map[string]struct{}{ ast.MD5: {}, ast.Reverse: {}, ast.VitessHash: {}, + ast.TiDBShard: {}, } diff --git a/sessiontxn/txn_context_test.go b/sessiontxn/txn_context_test.go index 97144263590b5..e9ec5f73dc466 100644 --- a/sessiontxn/txn_context_test.go +++ b/sessiontxn/txn_context_test.go @@ -34,7 +34,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/statistics/fmsketch.go b/statistics/fmsketch.go index a0e81c354ffe8..055956d03d01a 100644 --- a/statistics/fmsketch.go +++ b/statistics/fmsketch.go @@ -116,17 +116,6 @@ func (s *FMSketch) InsertRowValue(sc *stmtctx.StatementContext, values []types.D return nil } -func buildFMSketch(sc *stmtctx.StatementContext, values []types.Datum, maxSize int) (*FMSketch, int64, error) { - s := NewFMSketch(maxSize) - for _, value := range values { - err := s.InsertValue(sc, value) - if err != nil { - return nil, 0, errors.Trace(err) - } - } - return s, s.NDV(), nil -} - // MergeFMSketch merges two FM Sketch. func (s *FMSketch) MergeFMSketch(rs *FMSketch) { if s == nil || rs == nil { diff --git a/statistics/fmsketch_test.go b/statistics/fmsketch_test.go index 768db16fb913d..3b04b4dad8233 100644 --- a/statistics/fmsketch_test.go +++ b/statistics/fmsketch_test.go @@ -18,6 +18,7 @@ import ( "testing" "time" + "github.com/pingcap/errors" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/stretchr/testify/require" @@ -33,6 +34,17 @@ func extractSampleItemsDatums(items []*SampleItem) []types.Datum { return datums } +func buildFMSketch(sc *stmtctx.StatementContext, values []types.Datum, maxSize int) (*FMSketch, int64, error) { + s := NewFMSketch(maxSize) + for _, value := range values { + err := s.InsertValue(sc, value) + if err != nil { + return nil, 0, errors.Trace(err) + } + } + return s, s.NDV(), nil +} + func SubTestSketch() func(*testing.T) { return func(t *testing.T) { s := createTestStatisticsSamples(t) diff --git a/statistics/handle/ddl_test.go b/statistics/handle/ddl_test.go index 91a3a244cb17d..dfad7cbfbf5e8 100644 --- a/statistics/handle/ddl_test.go +++ b/statistics/handle/ddl_test.go @@ -26,8 +26,9 @@ import ( ) func TestDDLAfterLoad(t *testing.T) { - testKit, do, clean := createTestKitAndDom(t) + store, do, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") testKit.MustExec("analyze table t") @@ -59,8 +60,9 @@ func TestDDLAfterLoad(t *testing.T) { } func TestDDLTable(t *testing.T) { - testKit, do, clean := createTestKitAndDom(t) + store, do, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") is := do.InfoSchema() @@ -98,8 +100,9 @@ func TestDDLTable(t *testing.T) { } func TestDDLHistogram(t *testing.T) { - testKit, do, clean := createTestKitAndDom(t) + store, do, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) h := do.StatsHandle() testKit.MustExec("use test") @@ -185,8 +188,9 @@ func TestDDLHistogram(t *testing.T) { } func TestDDLPartition(t *testing.T) { - testKit, do, clean := createTestKitAndDom(t) + store, do, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index a3bcded88b7fb..50616f7c1b5f2 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -74,8 +74,9 @@ func cleanStats(tk *testkit.TestKit, do *domain.Domain) { } func TestConversion(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t (a int, b int)") @@ -123,8 +124,9 @@ func getStatsJSON(t *testing.T, dom *domain.Domain, db, tableName string) *handl } func TestDumpGlobalStats(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode = 'static'") @@ -149,8 +151,9 @@ func TestDumpGlobalStats(t *testing.T) { } func TestLoadGlobalStats(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") @@ -175,8 +178,9 @@ func TestLoadGlobalStats(t *testing.T) { } func TestDumpPartitions(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b int, primary key(a), index idx(b)) @@ -220,8 +224,9 @@ PARTITION BY RANGE ( a ) ( } func TestDumpAlteredTable(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") h := dom.StatsHandle() @@ -239,8 +244,9 @@ func TestDumpAlteredTable(t *testing.T) { func TestDumpCMSketchWithTopN(t *testing.T) { // Just test if we can store and recover the Top N elements stored in database. - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t(a int)") testKit.MustExec("insert into t values (1),(3),(4),(2),(5)") @@ -280,8 +286,9 @@ func TestDumpCMSketchWithTopN(t *testing.T) { } func TestDumpPseudoColumns(t *testing.T) { - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t(a int, b int, index idx(a))") // Force adding an pseudo tables in stats cache. @@ -297,8 +304,9 @@ func TestDumpPseudoColumns(t *testing.T) { } func TestDumpExtendedStats(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -332,8 +340,9 @@ func TestDumpExtendedStats(t *testing.T) { } func TestDumpVer2Stats(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -383,8 +392,9 @@ func TestDumpVer2Stats(t *testing.T) { } func TestJSONTableToBlocks(t *testing.T) { - tk, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/statistics/handle/gc_test.go b/statistics/handle/gc_test.go index b426613952c4d..c479c498dc36d 100644 --- a/statistics/handle/gc_test.go +++ b/statistics/handle/gc_test.go @@ -18,27 +18,15 @@ import ( "testing" "time" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" ) -func createTestKitAndDom(t *testing.T) (*testkit.TestKit, *domain.Domain, func()) { - store, dom, err := newStoreWithBootstrap() - require.NoError(t, err) - clean := func() { - dom.Close() - err := store.Close() - require.NoError(t, err) - } - tk := testkit.NewTestKit(t, store) - return tk, dom, clean -} - func TestGCStats(t *testing.T) { - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("set @@tidb_analyze_version = 1") testKit.MustExec("use test") testKit.MustExec("create table t(a int, b int, index idx(a, b), index idx_a(a))") @@ -70,8 +58,9 @@ func TestGCStats(t *testing.T) { } func TestGCPartition(t *testing.T) { - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("set @@tidb_analyze_version = 1") testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") @@ -108,8 +97,9 @@ func TestGCPartition(t *testing.T) { } func TestGCExtendedStats(t *testing.T) { - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("set session tidb_enable_extended_stats = on") testKit.MustExec("use test") testKit.MustExec("create table t(a int, b int, c int)") @@ -152,8 +142,9 @@ func TestGCExtendedStats(t *testing.T) { } func TestGCColumnStatsUsage(t *testing.T) { - testKit, dom, clean := createTestKitAndDom(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t(a int, b int, c int)") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 45db4f42df329..13e19d4b95eb0 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -74,7 +74,7 @@ type statsCache struct { // Handle can update stats info periodically. type Handle struct { mu struct { - sync.Mutex + sync.RWMutex ctx sessionctx.Context // rateMap contains the error rate delta from feedback. rateMap errorRateDeltaMap @@ -92,6 +92,7 @@ type Handle struct { memTracker *memory.Tracker } + // Deprecated: only used by feedback now pool sessionPool // ddlEventCh is a channel to notify a ddl operation has happened. @@ -120,56 +121,28 @@ type Handle struct { // idxUsageListHead contains all the index usage collectors required by session. idxUsageListHead *SessionIndexUsageCollector - // statsLoad is used to load stats concurrently + // StatsLoad is used to load stats concurrently StatsLoad StatsLoad } -func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context.Context, sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { - se, err := h.pool.Get() - if err != nil { - return nil, nil, errors.Trace(err) - } - defer h.pool.Put(se) - - exec := se.(sqlexec.RestrictedSQLExecutor) - return fn(ctx, exec) -} - func (h *Handle) execRestrictedSQL(ctx context.Context, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { - return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { - stmt, err := exec.ParseWithParams(ctx, true, sql, params...) - if err != nil { - return nil, nil, errors.Trace(err) - } - return exec.ExecRestrictedStmt(ctx, stmt) - }) + return h.mu.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseSessionPool}, sql, params...) } func (h *Handle) execRestrictedSQLWithStatsVer(ctx context.Context, statsVer int, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { - return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { - stmt, err := exec.ParseWithParams(ctx, true, sql, params...) - // TODO: An ugly way to set @@tidb_partition_prune_mode. Need to be improved. - if _, ok := stmt.(*ast.AnalyzeTableStmt); ok { - pruneMode := h.CurrentPruneMode() - if session, ok := exec.(sessionctx.Context); ok { - session.GetSessionVars().PartitionPruneMode.Store(string(pruneMode)) - } - } - if err != nil { - return nil, nil, errors.Trace(err) - } - return exec.ExecRestrictedStmt(ctx, stmt, execOptionForAnalyze[statsVer]) - }) + optFuncs := []sqlexec.OptionFuncAlias{ + sqlexec.ExecOptionUseSessionPool, + execOptionForAnalyze[statsVer], + } + return h.mu.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, optFuncs, sql, params...) } func (h *Handle) execRestrictedSQLWithSnapshot(ctx context.Context, sql string, snapshot uint64, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { - return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { - stmt, err := exec.ParseWithParams(ctx, true, sql, params...) - if err != nil { - return nil, nil, errors.Trace(err) - } - return exec.ExecRestrictedStmt(ctx, stmt, sqlexec.ExecOptionWithSnapshot(snapshot)) - }) + optFuncs := []sqlexec.OptionFuncAlias{ + sqlexec.ExecOptionUseSessionPool, + sqlexec.ExecOptionWithSnapshot(snapshot), + } + return h.mu.ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, optFuncs, sql, params...) } // Clear the statsCache, only for test. @@ -1403,14 +1376,10 @@ type statsReader struct { func (sr *statsReader) read(sql string, args ...interface{}) (rows []chunk.Row, fields []*ast.ResultField, err error) { ctx := context.TODO() - stmt, err := sr.ctx.ParseWithParams(ctx, true, sql, args...) - if err != nil { - return nil, nil, errors.Trace(err) - } if sr.snapshot > 0 { - return sr.ctx.ExecRestrictedStmt(ctx, stmt, sqlexec.ExecOptionWithSnapshot(sr.snapshot)) + return sr.ctx.ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseSessionPool, sqlexec.ExecOptionWithSnapshot(sr.snapshot)}, sql, args...) } - return sr.ctx.ExecRestrictedStmt(ctx, stmt) + return sr.ctx.ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseCurSession}, sql, args...) } func (sr *statsReader) isHistory() bool { @@ -1793,8 +1762,6 @@ func (h *Handle) SaveExtendedStatsToStorage(tableID int64, extStats *statistics. // CurrentPruneMode indicates whether tbl support runtime prune for table and first partition id. func (h *Handle) CurrentPruneMode() variable.PartitionPruneMode { - h.mu.Lock() - defer h.mu.Unlock() return variable.PartitionPruneMode(h.mu.ctx.GetSessionVars().PartitionPruneMode.Load()) } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index ed151a08f2310..e5c0a97d1def5 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -24,86 +24,59 @@ import ( "time" "unsafe" - . "github.com/pingcap/check" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" - "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/testkit" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) -func TestT(t *testing.T) { - TestingT(t) -} - -// TODO replace cleanEnv with createTestKitAndDom in gc_series_test.go when migrate this file -func cleanEnv(c *C, store kv.Storage, do *domain.Domain) { - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } - tk.MustExec("delete from mysql.stats_meta") - tk.MustExec("delete from mysql.stats_histograms") - tk.MustExec("delete from mysql.stats_buckets") - tk.MustExec("delete from mysql.stats_extended") - tk.MustExec("delete from mysql.stats_fm_sketch") - tk.MustExec("delete from mysql.schema_index_usage") - tk.MustExec("delete from mysql.column_stats_usage") - tk.MustExec("delete from mysql.tidb where variable_name = 'tidb_disable_column_tracking_time'") - do.StatsHandle().Clear() -} - -func (s *testStatsSuite) TestStatsCache(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestStatsCache(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") testKit.MustExec("insert into t values(1, 2)") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsTrue) + require.True(t, statsTbl.Pseudo) testKit.MustExec("analyze table t") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) testKit.MustExec("create index idx_t on t(c1)") do.InfoSchema() statsTbl = do.StatsHandle().GetTableStats(tableInfo) // If index is build, but stats is not updated. statsTbl can also work. - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) // But the added index will not work. - c.Assert(statsTbl.Indices[int64(1)], IsNil) + require.Nil(t, statsTbl.Indices[int64(1)]) testKit.MustExec("analyze table t") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) // If the new schema drop a column, the table stats can still work. testKit.MustExec("alter table t drop column c2") is = do.InfoSchema() do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) // If the new schema add a column, the table stats can still work. testKit.MustExec("alter table t add column c10 int") @@ -111,54 +84,55 @@ func (s *testStatsSuite) TestStatsCache(c *C) { do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) } -func (s *testStatsSuite) TestStatsCacheMemTracker(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestStatsCacheMemTracker(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int,c3 int)") testKit.MustExec("insert into t values(1, 2, 3)") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.MemoryUsage() > 0, IsTrue) - c.Assert(statsTbl.Pseudo, IsTrue) + require.True(t, statsTbl.MemoryUsage() > 0) + require.True(t, statsTbl.Pseudo) testKit.MustExec("analyze table t") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) testKit.MustExec("create index idx_t on t(c1)") do.InfoSchema() statsTbl = do.StatsHandle().GetTableStats(tableInfo) // If index is build, but stats is not updated. statsTbl can also work. - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) // But the added index will not work. - c.Assert(statsTbl.Indices[int64(1)], IsNil) + require.Nil(t, statsTbl.Indices[int64(1)]) testKit.MustExec("analyze table t") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) // If the new schema drop a column, the table stats can still work. testKit.MustExec("alter table t drop column c2") is = do.InfoSchema() do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.MemoryUsage() > 0, IsTrue) - c.Assert(statsTbl.Pseudo, IsFalse) + require.True(t, statsTbl.MemoryUsage() > 0) + require.False(t, statsTbl.Pseudo) // If the new schema add a column, the table stats can still work. testKit.MustExec("alter table t add column c10 int") @@ -166,37 +140,37 @@ func (s *testStatsSuite) TestStatsCacheMemTracker(c *C) { do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Pseudo, IsFalse) + require.False(t, statsTbl.Pseudo) } -func assertTableEqual(c *C, a *statistics.Table, b *statistics.Table) { - c.Assert(a.Count, Equals, b.Count) - c.Assert(a.ModifyCount, Equals, b.ModifyCount) - c.Assert(len(a.Columns), Equals, len(b.Columns)) +func assertTableEqual(t *testing.T, a *statistics.Table, b *statistics.Table) { + require.Equal(t, b.Count, a.Count) + require.Equal(t, b.ModifyCount, a.ModifyCount) + require.Len(t, a.Columns, len(b.Columns)) for i := range a.Columns { - c.Assert(a.Columns[i].Count, Equals, b.Columns[i].Count) - c.Assert(statistics.HistogramEqual(&a.Columns[i].Histogram, &b.Columns[i].Histogram, false), IsTrue) + require.Equal(t, b.Columns[i].Count, a.Columns[i].Count) + require.True(t, statistics.HistogramEqual(&a.Columns[i].Histogram, &b.Columns[i].Histogram, false)) if a.Columns[i].CMSketch == nil { - c.Assert(b.Columns[i].CMSketch, IsNil) + require.Nil(t, b.Columns[i].CMSketch) } else { - c.Assert(a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch), IsTrue) + require.True(t, a.Columns[i].CMSketch.Equal(b.Columns[i].CMSketch)) } // The nil case has been considered in (*TopN).Equal() so we don't need to consider it here. - c.Assert(a.Columns[i].TopN.Equal(b.Columns[i].TopN), IsTrue, Commentf("%v, %v", a.Columns[i].TopN, b.Columns[i].TopN)) + require.Truef(t, a.Columns[i].TopN.Equal(b.Columns[i].TopN), "%v, %v", a.Columns[i].TopN, b.Columns[i].TopN) } - c.Assert(len(a.Indices), Equals, len(b.Indices)) + require.Len(t, a.Indices, len(b.Indices)) for i := range a.Indices { - c.Assert(statistics.HistogramEqual(&a.Indices[i].Histogram, &b.Indices[i].Histogram, false), IsTrue) + require.True(t, statistics.HistogramEqual(&a.Indices[i].Histogram, &b.Indices[i].Histogram, false)) if a.Indices[i].CMSketch == nil { - c.Assert(b.Indices[i].CMSketch, IsNil) + require.Nil(t, b.Indices[i].CMSketch) } else { - c.Assert(a.Indices[i].CMSketch.Equal(b.Indices[i].CMSketch), IsTrue) + require.True(t, a.Indices[i].CMSketch.Equal(b.Indices[i].CMSketch)) } - c.Assert(a.Indices[i].TopN.Equal(b.Indices[i].TopN), IsTrue) + require.True(t, a.Indices[i].TopN.Equal(b.Indices[i].TopN)) } - c.Assert(isSameExtendedStats(a.ExtendedStats, b.ExtendedStats), IsTrue) + require.True(t, isSameExtendedStats(a.ExtendedStats, b.ExtendedStats)) } func isSameExtendedStats(a, b *statistics.ExtendedStatsColl) bool { @@ -228,9 +202,10 @@ func isSameExtendedStats(a, b *statistics.ExtendedStatsColl) bool { return true } -func (s *testStatsSuite) TestStatsStoreAndLoad(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestStatsStoreAndLoad(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") recordCount := 1000 @@ -238,10 +213,10 @@ func (s *testStatsSuite) TestStatsStoreAndLoad(c *C) { testKit.MustExec("insert into t values (?, ?)", i, i+1) } testKit.MustExec("create index idx_t on t(c2)") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() testKit.MustExec("analyze table t") @@ -249,40 +224,42 @@ func (s *testStatsSuite) TestStatsStoreAndLoad(c *C) { do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl2 := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl2.Pseudo, IsFalse) - c.Assert(statsTbl2.Count, Equals, int64(recordCount)) - assertTableEqual(c, statsTbl1, statsTbl2) + require.False(t, statsTbl2.Pseudo) + require.Equal(t, int64(recordCount), statsTbl2.Count) + assertTableEqual(t, statsTbl1, statsTbl2) } -func (s *testStatsSuite) TestEmptyTable(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestEmptyTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int, key cc1(c1), key cc2(c2))") testKit.MustExec("analyze table t") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) count := statsTbl.ColumnGreaterRowCount(mock.NewContext(), types.NewDatum(1), tableInfo.Columns[0].ID) - c.Assert(count, Equals, 0.0) + require.Equal(t, 0.0, count) } -func (s *testStatsSuite) TestColumnIDs(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestColumnIDs(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") testKit.MustExec("insert into t values(1, 2)") testKit.MustExec("analyze table t") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) sctx := mock.NewContext() @@ -294,168 +271,171 @@ func (s *testStatsSuite) TestColumnIDs(c *C) { Collators: collate.GetBinaryCollatorSlice(1), } count, err := statsTbl.GetRowCountByColumnRanges(sctx, tableInfo.Columns[0].ID, []*ranger.Range{ran}) - c.Assert(err, IsNil) - c.Assert(count, Equals, float64(1)) + require.NoError(t, err) + require.Equal(t, float64(1), count) // Drop a column and the offset changed, testKit.MustExec("alter table t drop column c1") is = do.InfoSchema() do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) // At that time, we should get c2's stats instead of c1's. count, err = statsTbl.GetRowCountByColumnRanges(sctx, tableInfo.Columns[0].ID, []*ranger.Range{ran}) - c.Assert(err, IsNil) - c.Assert(count, Equals, 0.0) + require.NoError(t, err) + require.Equal(t, 0.0, count) } -func (s *testStatsSuite) TestAvgColLen(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestAvgColLen(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 varchar(100), c3 float, c4 datetime, c5 varchar(100))") testKit.MustExec("insert into t values(1, '1234567', 12.3, '2018-03-07 19:00:57', NULL)") testKit.MustExec("analyze table t") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0) + require.Equal(t, 1.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count)) // The size of varchar type is LEN + BYTE, here is 1 + 7 = 8 - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0-3) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3)))) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.ZeroTime))) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0-3+8) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3)))) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.ZeroTime))) - c.Assert(statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 0.0) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0-3, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, 8.0-3+8, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, 0.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.Count)) testKit.MustExec("insert into t values(132, '123456789112', 1232.3, '2018-03-07 19:17:29', NULL)") testKit.MustExec("analyze table t") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false), Equals, 1.5) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false), Equals, 10.5) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, math.Round((10.5-math.Log2(10.5))*100)/100) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3)))) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.ZeroTime))) - c.Assert(statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, math.Round((10.5-math.Log2(10.5))*100)/100+8) - c.Assert(statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(float32(12.3)))) - c.Assert(statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, float64(unsafe.Sizeof(types.ZeroTime))) - c.Assert(statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.Count), Equals, 8.0) - c.Assert(statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.Count), Equals, 0.0) -} - -func (s *testStatsSuite) TestDurationToTS(c *C) { + require.Equal(t, 1.5, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 10.5, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSize(statsTbl.Count, false)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeListInDisk(statsTbl.Count)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[0].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, math.Round((10.5-math.Log2(10.5))*100)/100+8, statsTbl.Columns[tableInfo.Columns[1].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(float32(12.3))), statsTbl.Columns[tableInfo.Columns[2].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, float64(unsafe.Sizeof(types.ZeroTime)), statsTbl.Columns[tableInfo.Columns[3].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, 8.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeChunkFormat(statsTbl.Count)) + require.Equal(t, 0.0, statsTbl.Columns[tableInfo.Columns[4].ID].AvgColSizeListInDisk(statsTbl.Count)) +} + +func TestDurationToTS(t *testing.T) { tests := []time.Duration{time.Millisecond, time.Second, time.Minute, time.Hour} - for _, t := range tests { - ts := handle.DurationToTS(t) - c.Assert(oracle.ExtractPhysical(ts)*int64(time.Millisecond), Equals, int64(t)) + for _, test := range tests { + ts := handle.DurationToTS(test) + require.Equal(t, int64(test), oracle.ExtractPhysical(ts)*int64(time.Millisecond)) } } -func (s *testStatsSuite) TestVersion(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestVersion(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int, c2 int)") testKit.MustExec("analyze table t1") - do := s.do + do := dom is := do.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo1 := tbl1.Meta() - h, err := handle.NewHandle(testKit.Se, time.Millisecond, do.SysSessionPool()) - c.Assert(err, IsNil) + h, err := handle.NewHandle(testKit.Session(), time.Millisecond, do.SysSessionPool()) + require.NoError(t, err) unit := oracle.ComposeTS(1, 0) testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", 2*unit, tableInfo1.ID) - c.Assert(h.Update(is), IsNil) - c.Assert(h.LastUpdateVersion(), Equals, 2*unit) + require.NoError(t, h.Update(is)) + require.Equal(t, 2*unit, h.LastUpdateVersion()) statsTbl1 := h.GetTableStats(tableInfo1) - c.Assert(statsTbl1.Pseudo, IsFalse) + require.False(t, statsTbl1.Pseudo) testKit.MustExec("create table t2 (c1 int, c2 int)") testKit.MustExec("analyze table t2") is = do.InfoSchema() tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo2 := tbl2.Meta() // A smaller version write, and we can still read it. testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", unit, tableInfo2.ID) - c.Assert(h.Update(is), IsNil) - c.Assert(h.LastUpdateVersion(), Equals, 2*unit) + require.NoError(t, h.Update(is)) + require.Equal(t, 2*unit, h.LastUpdateVersion()) statsTbl2 := h.GetTableStats(tableInfo2) - c.Assert(statsTbl2.Pseudo, IsFalse) + require.False(t, statsTbl2.Pseudo) testKit.MustExec("insert t1 values(1,2)") testKit.MustExec("analyze table t1") offset := 3 * unit testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", offset+4, tableInfo1.ID) - c.Assert(h.Update(is), IsNil) - c.Assert(h.LastUpdateVersion(), Equals, offset+uint64(4)) + require.NoError(t, h.Update(is)) + require.Equal(t, offset+uint64(4), h.LastUpdateVersion()) statsTbl1 = h.GetTableStats(tableInfo1) - c.Assert(statsTbl1.Count, Equals, int64(1)) + require.Equal(t, int64(1), statsTbl1.Count) testKit.MustExec("insert t2 values(1,2)") testKit.MustExec("analyze table t2") // A smaller version write, and we can still read it. testKit.MustExec("update mysql.stats_meta set version = ? where table_id = ?", offset+3, tableInfo2.ID) - c.Assert(h.Update(is), IsNil) - c.Assert(h.LastUpdateVersion(), Equals, offset+uint64(4)) + require.NoError(t, h.Update(is)) + require.Equal(t, offset+uint64(4), h.LastUpdateVersion()) statsTbl2 = h.GetTableStats(tableInfo2) - c.Assert(statsTbl2.Count, Equals, int64(1)) + require.Equal(t, int64(1), statsTbl2.Count) testKit.MustExec("insert t2 values(1,2)") testKit.MustExec("analyze table t2") // A smaller version write, and we cannot read it. Because at this time, lastThree Version is 4. testKit.MustExec("update mysql.stats_meta set version = 1 where table_id = ?", tableInfo2.ID) - c.Assert(h.Update(is), IsNil) - c.Assert(h.LastUpdateVersion(), Equals, offset+uint64(4)) + require.NoError(t, h.Update(is)) + require.Equal(t, offset+uint64(4), h.LastUpdateVersion()) statsTbl2 = h.GetTableStats(tableInfo2) - c.Assert(statsTbl2.Count, Equals, int64(1)) + require.Equal(t, int64(1), statsTbl2.Count) // We add an index and analyze it, but DDL doesn't load. testKit.MustExec("alter table t2 add column c3 int") testKit.MustExec("analyze table t2") // load it with old schema. - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl2 = h.GetTableStats(tableInfo2) - c.Assert(statsTbl2.Pseudo, IsFalse) - c.Assert(statsTbl2.Columns[int64(3)], IsNil) + require.False(t, statsTbl2.Pseudo) + require.Nil(t, statsTbl2.Columns[int64(3)]) // Next time DDL updated. is = do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl2 = h.GetTableStats(tableInfo2) - c.Assert(statsTbl2.Pseudo, IsFalse) + require.False(t, statsTbl2.Pseudo) // We can read it without analyze again! Thanks for PrevLastVersion. - c.Assert(statsTbl2.Columns[int64(3)], NotNil) + require.NotNil(t, statsTbl2.Columns[int64(3)]) } -func (s *testStatsSuite) TestLoadHist(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestLoadHist(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (c1 varchar(12), c2 char(12))") - do := s.do + do := dom h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) rowCount := 10 for i := 0; i < rowCount; i++ { testKit.MustExec("insert into t values('a','ddd')") @@ -463,197 +443,190 @@ func (s *testStatsSuite) TestLoadHist(c *C) { testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() oldStatsTbl := h.GetTableStats(tableInfo) for i := 0; i < rowCount; i++ { testKit.MustExec("insert into t values('bb','sdfga')") } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) err = h.Update(do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, err) newStatsTbl := h.GetTableStats(tableInfo) // The stats table is updated. - c.Assert(oldStatsTbl == newStatsTbl, IsFalse) + require.False(t, oldStatsTbl == newStatsTbl) // Only the TotColSize of histograms is updated. for id, hist := range oldStatsTbl.Columns { - c.Assert(hist.TotColSize, Less, newStatsTbl.Columns[id].TotColSize) + require.Less(t, hist.TotColSize, newStatsTbl.Columns[id].TotColSize) temp := hist.TotColSize hist.TotColSize = newStatsTbl.Columns[id].TotColSize - c.Assert(statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.Columns[id].Histogram, false), IsTrue) + require.True(t, statistics.HistogramEqual(&hist.Histogram, &newStatsTbl.Columns[id].Histogram, false)) hist.TotColSize = temp - c.Assert(hist.CMSketch.Equal(newStatsTbl.Columns[id].CMSketch), IsTrue) - c.Assert(hist.Count, Equals, newStatsTbl.Columns[id].Count) - c.Assert(hist.Info, Equals, newStatsTbl.Columns[id].Info) + require.True(t, hist.CMSketch.Equal(newStatsTbl.Columns[id].CMSketch)) + require.Equal(t, newStatsTbl.Columns[id].Count, hist.Count) + require.Equal(t, newStatsTbl.Columns[id].Info, hist.Info) } // Add column c3, we only update c3. testKit.MustExec("alter table t add column c3 int") err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) is = do.InfoSchema() tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo = tbl.Meta() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) newStatsTbl2 := h.GetTableStats(tableInfo) - c.Assert(newStatsTbl2 == newStatsTbl, IsFalse) + require.False(t, newStatsTbl2 == newStatsTbl) // The histograms is not updated. for id, hist := range newStatsTbl.Columns { - c.Assert(hist, Equals, newStatsTbl2.Columns[id]) + require.Equal(t, newStatsTbl2.Columns[id], hist) } - c.Assert(newStatsTbl2.Columns[int64(3)].LastUpdateVersion, Greater, newStatsTbl2.Columns[int64(1)].LastUpdateVersion) + require.Greater(t, newStatsTbl2.Columns[int64(3)].LastUpdateVersion, newStatsTbl2.Columns[int64(1)].LastUpdateVersion) } -func (s *testStatsSuite) TestInitStats(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestInitStats(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 1") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,7,8)") testKit.MustExec("analyze table t") - h := s.do.StatsHandle() - is := s.do.InfoSchema() + h := dom.StatsHandle() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) // `Update` will not use load by need strategy when `Lease` is 0, and `InitStats` is only called when // `Lease` is not 0, so here we just change it. h.SetLease(time.Millisecond) h.Clear() - c.Assert(h.InitStats(is), IsNil) + require.NoError(t, h.InitStats(is)) table0 := h.GetTableStats(tbl.Meta()) cols := table0.Columns - c.Assert(cols[1].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x36)) - c.Assert(cols[2].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x37)) - c.Assert(cols[3].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x38)) + require.Equal(t, uint8(0x36), cols[1].LastAnalyzePos.GetBytes()[0]) + require.Equal(t, uint8(0x37), cols[2].LastAnalyzePos.GetBytes()[0]) + require.Equal(t, uint8(0x38), cols[3].LastAnalyzePos.GetBytes()[0]) h.Clear() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) table1 := h.GetTableStats(tbl.Meta()) - assertTableEqual(c, table0, table1) + assertTableEqual(t, table0, table1) h.SetLease(0) } -func (s *testStatsSuite) TestInitStatsVer2(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestInitStatsVer2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("create table t(a int, b int, c int, index idx(a), index idxab(a, b))") tk.MustExec("insert into t values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (4, 4, 4), (4, 4, 4)") tk.MustExec("analyze table t with 2 topn, 3 buckets") - h := s.do.StatsHandle() - is := s.do.InfoSchema() + h := dom.StatsHandle() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) // `Update` will not use load by need strategy when `Lease` is 0, and `InitStats` is only called when // `Lease` is not 0, so here we just change it. h.SetLease(time.Millisecond) h.Clear() - c.Assert(h.InitStats(is), IsNil) + require.NoError(t, h.InitStats(is)) table0 := h.GetTableStats(tbl.Meta()) cols := table0.Columns - c.Assert(cols[1].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x33)) - c.Assert(cols[2].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x33)) - c.Assert(cols[3].LastAnalyzePos.GetBytes()[0], Equals, uint8(0x33)) + require.Equal(t, uint8(0x33), cols[1].LastAnalyzePos.GetBytes()[0]) + require.Equal(t, uint8(0x33), cols[2].LastAnalyzePos.GetBytes()[0]) + require.Equal(t, uint8(0x33), cols[3].LastAnalyzePos.GetBytes()[0]) h.Clear() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) table1 := h.GetTableStats(tbl.Meta()) - assertTableEqual(c, table0, table1) + assertTableEqual(t, table0, table1) h.SetLease(0) } -func (s *testStatsSuite) TestReloadExtStatsLockRelease(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestReloadExtStatsLockRelease(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values(1,1),(2,2),(3,3)") tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr", `return("")`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr", `return("")`)) err := tk.ExecToErr("admin reload stats_extended") - c.Assert(err.Error(), Equals, "gofail extendedStatsFromStorage error") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr"), IsNil) + require.Equal(t, "gofail extendedStatsFromStorage error", err.Error()) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/handle/injectExtStatsLoadErr")) // Check the lock is released by `admin reload stats_extended` if error happens. tk.MustExec("analyze table t") } -func (s *testStatsSuite) TestLoadStats(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestLoadStats(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") testKit.MustExec("insert into t values (1,1,1),(2,2,2),(3,3,3)") - oriLease := s.do.StatsHandle().Lease() - s.do.StatsHandle().SetLease(1) + oriLease := dom.StatsHandle().Lease() + dom.StatsHandle().SetLease(1) defer func() { - s.do.StatsHandle().SetLease(oriLease) + dom.StatsHandle().SetLease(oriLease) }() testKit.MustExec("analyze table t") - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() stat := h.GetTableStats(tableInfo) hg := stat.Columns[tableInfo.Columns[0].ID].Histogram - c.Assert(hg.Len(), Greater, 0) + require.Greater(t, hg.Len(), 0) cms := stat.Columns[tableInfo.Columns[0].ID].CMSketch - c.Assert(cms, IsNil) + require.Nil(t, cms) hg = stat.Indices[tableInfo.Indices[0].ID].Histogram - c.Assert(hg.Len(), Greater, 0) + require.Greater(t, hg.Len(), 0) cms = stat.Indices[tableInfo.Indices[0].ID].CMSketch topN := stat.Indices[tableInfo.Indices[0].ID].TopN - c.Assert(cms.TotalCount()+topN.TotalCount(), Greater, uint64(0)) + require.Greater(t, cms.TotalCount()+topN.TotalCount(), uint64(0)) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram - c.Assert(hg.Len(), Equals, 0) + require.Equal(t, 0, hg.Len()) cms = stat.Columns[tableInfo.Columns[2].ID].CMSketch - c.Assert(cms, IsNil) - _, err = stat.ColumnEqualRowCount(testKit.Se, types.NewIntDatum(1), tableInfo.Columns[2].ID) - c.Assert(err, IsNil) - c.Assert(h.LoadNeededHistograms(), IsNil) + require.Nil(t, cms) + _, err = stat.ColumnEqualRowCount(testKit.Session(), types.NewIntDatum(1), tableInfo.Columns[2].ID) + require.NoError(t, err) + require.NoError(t, h.LoadNeededHistograms()) stat = h.GetTableStats(tableInfo) hg = stat.Columns[tableInfo.Columns[2].ID].Histogram - c.Assert(hg.Len(), Greater, 0) + require.Greater(t, hg.Len(), 0) // Following test tests whether the LoadNeededHistograms would panic. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderFail", `return(true)`), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderFail", `return(true)`)) err = h.LoadNeededHistograms() - c.Assert(err, NotNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderFail"), IsNil) + require.Error(t, err) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderFail")) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic", "panic"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic", "panic")) err = h.LoadNeededHistograms() - c.Assert(err, ErrorMatches, ".*getStatsReader panic.*") - c.Assert(failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic"), IsNil) + require.Error(t, err) + require.Regexp(t, ".*getStatsReader panic.*", err.Error()) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/handle/mockGetStatsReaderPanic")) err = h.LoadNeededHistograms() - c.Assert(err, IsNil) + require.NoError(t, err) } -func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { - store, err := mockstore.NewMockStore() - if err != nil { - return nil, nil, errors.Trace(err) - } - session.SetSchemaLease(0) - session.DisableStats4Test() - domain.RunAutoAnalyze = false - do, err := session.BootstrapSession(store) - do.SetStatsUpdating(true) - return store, do, errors.Trace(err) -} - -func (s *testStatsSuite) TestCorrelation(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestCorrelation(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t(c1 int primary key, c2 int)") testKit.MustExec("select * from t where c1 > 10 and c2 > 10") @@ -661,73 +634,73 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result := testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("insert into t values(8,18)") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) testKit.MustExec("truncate table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) testKit.MustExec("insert into t values(1,21),(3,12),(4,7),(2,20),(5,1)") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "-1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "-1", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "-1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "-1", result.Rows()[1][9]) testKit.MustExec("insert into t values(8,4)") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "-0.9428571428571428") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "-0.9428571428571428", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "-0.9428571428571428") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "-0.9428571428571428", result.Rows()[1][9]) testKit.MustExec("truncate table t") testKit.MustExec("insert into t values (1,1),(2,1),(3,1),(4,1),(5,1),(6,1),(7,1),(8,1),(9,1),(10,1),(11,1),(12,1),(13,1),(14,1),(15,1),(16,1),(17,1),(18,1),(19,1),(20,2),(21,2),(22,2),(23,2),(24,2),(25,2)") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("drop table t") testKit.MustExec("create table t(c1 int, c2 int)") @@ -735,30 +708,30 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") + require.Len(t, result.Rows(), 2) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "0.8285714285714286", result.Rows()[1][9]) testKit.MustExec("truncate table t") testKit.MustExec("insert into t values(1,1),(2,7),(3,12),(8,18),(4,20),(5,21)") testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0.8285714285714286") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0.8285714285714286", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0.8285714285714286") - c.Assert(result.Rows()[1][9], Equals, "1") + require.Len(t, result.Rows(), 2) + require.Equal(t, "0.8285714285714286", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) testKit.MustExec("drop table t") testKit.MustExec("create table t(c1 int primary key, c2 int, c3 int, key idx_c2(c2))") @@ -766,40 +739,42 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() - c.Assert(len(result.Rows()), Equals, 3) - c.Assert(result.Rows()[0][9], Equals, "0") - c.Assert(result.Rows()[1][9], Equals, "1") - c.Assert(result.Rows()[2][9], Equals, "1") + require.Len(t, result.Rows(), 3) + require.Equal(t, "0", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + require.Equal(t, "1", result.Rows()[2][9]) result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][9], Equals, "0") + require.Len(t, result.Rows(), 1) + require.Equal(t, "0", result.Rows()[0][9]) testKit.MustExec("set @@tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() - c.Assert(len(result.Rows()), Equals, 3) - c.Assert(result.Rows()[0][9], Equals, "1") - c.Assert(result.Rows()[1][9], Equals, "1") - c.Assert(result.Rows()[2][9], Equals, "1") + require.Len(t, result.Rows(), 3) + require.Equal(t, "1", result.Rows()[0][9]) + require.Equal(t, "1", result.Rows()[1][9]) + require.Equal(t, "1", result.Rows()[2][9]) result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][9], Equals, "0") + require.Len(t, result.Rows(), 1) + require.Equal(t, "0", result.Rows()[0][9]) } -func (s *testStatsSuite) TestAnalyzeVirtualCol(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeVirtualCol(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int generated always as (-a) virtual, c int generated always as (-a) stored, index (c))") tk.MustExec("insert into t(a) values(2),(1),(1),(3),(NULL)") tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("analyze table t") - c.Assert(len(tk.MustQuery("show stats_histograms where table_name ='t'").Rows()), Equals, 3) + require.Len(t, tk.MustQuery("show stats_histograms where table_name ='t'").Rows(), 3) } -func (s *testStatsSuite) TestShowGlobalStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestShowGlobalStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 0") tk.MustExec("drop table if exists t") @@ -807,31 +782,32 @@ func (s *testStatsSuite) TestShowGlobalStats(c *C) { tk.MustExec("create table t (a int, key(a)) partition by hash(a) partitions 2") tk.MustExec("insert into t values (1), (2), (3), (4)") tk.MustExec("analyze table t with 1 buckets") - c.Assert(len(tk.MustQuery("show stats_meta").Rows()), Equals, 2) - c.Assert(len(tk.MustQuery("show stats_meta where partition_name='global'").Rows()), Equals, 0) - c.Assert(len(tk.MustQuery("show stats_buckets").Rows()), Equals, 4) // 2 partitions * (1 for the column_a and 1 for the index_a) - c.Assert(len(tk.MustQuery("show stats_buckets where partition_name='global'").Rows()), Equals, 0) - c.Assert(len(tk.MustQuery("show stats_histograms").Rows()), Equals, 4) - c.Assert(len(tk.MustQuery("show stats_histograms where partition_name='global'").Rows()), Equals, 0) - c.Assert(len(tk.MustQuery("show stats_healthy").Rows()), Equals, 2) - c.Assert(len(tk.MustQuery("show stats_healthy where partition_name='global'").Rows()), Equals, 0) + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_meta where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_buckets").Rows(), 4) // 2 partitions * (1 for the column_a and 1 for the index_a) + require.Len(t, tk.MustQuery("show stats_buckets where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_histograms").Rows(), 4) + require.Len(t, tk.MustQuery("show stats_histograms where partition_name='global'").Rows(), 0) + require.Len(t, tk.MustQuery("show stats_healthy").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_healthy where partition_name='global'").Rows(), 0) tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec("analyze table t with 0 topn, 1 buckets") - c.Assert(len(tk.MustQuery("show stats_meta").Rows()), Equals, 3) - c.Assert(len(tk.MustQuery("show stats_meta where partition_name='global'").Rows()), Equals, 1) - c.Assert(len(tk.MustQuery("show stats_buckets").Rows()), Equals, 6) - c.Assert(len(tk.MustQuery("show stats_buckets where partition_name='global'").Rows()), Equals, 2) - c.Assert(len(tk.MustQuery("show stats_histograms").Rows()), Equals, 6) - c.Assert(len(tk.MustQuery("show stats_histograms where partition_name='global'").Rows()), Equals, 2) - c.Assert(len(tk.MustQuery("show stats_healthy").Rows()), Equals, 3) - c.Assert(len(tk.MustQuery("show stats_healthy where partition_name='global'").Rows()), Equals, 1) -} - -func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 3) + require.Len(t, tk.MustQuery("show stats_meta where partition_name='global'").Rows(), 1) + require.Len(t, tk.MustQuery("show stats_buckets").Rows(), 6) + require.Len(t, tk.MustQuery("show stats_buckets where partition_name='global'").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_histograms").Rows(), 6) + require.Len(t, tk.MustQuery("show stats_histograms where partition_name='global'").Rows(), 2) + require.Len(t, tk.MustQuery("show stats_healthy").Rows(), 3) + require.Len(t, tk.MustQuery("show stats_healthy where partition_name='global'").Rows(), 1) +} + +func TestBuildGlobalLevelStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t, t1;") testKit.MustExec("set @@tidb_analyze_version = 2") @@ -844,50 +820,50 @@ func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { testKit.MustExec("create index idx_t_b on t(b);") testKit.MustExec("analyze table t, t1;") result := testKit.MustQuery("show stats_meta where table_name = 't';").Sort() - c.Assert(len(result.Rows()), Equals, 3) - c.Assert(result.Rows()[0][5], Equals, "1") - c.Assert(result.Rows()[1][5], Equals, "2") - c.Assert(result.Rows()[2][5], Equals, "2") + require.Len(t, result.Rows(), 3) + require.Equal(t, "1", result.Rows()[0][5]) + require.Equal(t, "2", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() - c.Assert(len(result.Rows()), Equals, 15) + require.Len(t, result.Rows(), 15) result = testKit.MustQuery("show stats_meta where table_name = 't1';").Sort() - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][5], Equals, "5") + require.Len(t, result.Rows(), 1) + require.Equal(t, "5", result.Rows()[0][5]) result = testKit.MustQuery("show stats_histograms where table_name = 't1';").Sort() - c.Assert(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) // Test the 'dynamic' mode testKit.MustExec("set @@tidb_partition_prune_mode = 'dynamic';") testKit.MustExec("analyze table t, t1;") result = testKit.MustQuery("show stats_meta where table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 4) - c.Assert(result.Rows()[0][5], Equals, "5") - c.Assert(result.Rows()[1][5], Equals, "1") - c.Assert(result.Rows()[2][5], Equals, "2") - c.Assert(result.Rows()[3][5], Equals, "2") + require.Len(t, result.Rows(), 4) + require.Equal(t, "5", result.Rows()[0][5]) + require.Equal(t, "1", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) + require.Equal(t, "2", result.Rows()[3][5]) result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() - c.Assert(len(result.Rows()), Equals, 20) + require.Len(t, result.Rows(), 20) result = testKit.MustQuery("show stats_meta where table_name = 't1';").Sort() - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][5], Equals, "5") + require.Len(t, result.Rows(), 1) + require.Equal(t, "5", result.Rows()[0][5]) result = testKit.MustQuery("show stats_histograms where table_name = 't1';").Sort() - c.Assert(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) testKit.MustExec("analyze table t index idx_t_ab, idx_t_b;") result = testKit.MustQuery("show stats_meta where table_name = 't'").Sort() - c.Assert(len(result.Rows()), Equals, 4) - c.Assert(result.Rows()[0][5], Equals, "5") - c.Assert(result.Rows()[1][5], Equals, "1") - c.Assert(result.Rows()[2][5], Equals, "2") - c.Assert(result.Rows()[3][5], Equals, "2") + require.Len(t, result.Rows(), 4) + require.Equal(t, "5", result.Rows()[0][5]) + require.Equal(t, "1", result.Rows()[1][5]) + require.Equal(t, "2", result.Rows()[2][5]) + require.Equal(t, "2", result.Rows()[3][5]) result = testKit.MustQuery("show stats_histograms where table_name = 't';").Sort() - c.Assert(len(result.Rows()), Equals, 20) + require.Len(t, result.Rows(), 20) } // nolint:unused -func (s *testSerialStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, tblName, dbName string) { +func prepareForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, tk *testkit.TestKit, tblName, dbName string) { tk.MustExec("create database if not exists " + dbName) tk.MustExec("use " + dbName) tk.MustExec("drop table if exists " + tblName) @@ -907,25 +883,25 @@ func (s *testSerialStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.T tk.MustExec(buf2.String()) tk.MustExec("set @@tidb_analyze_version=2") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) } // nolint:unused -func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, db, t, p string, topn, buckets int) { - tbl, err := s.do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(t)) - c.Assert(err, IsNil) +func checkForGlobalStatsWithOpts(t *testing.T, dom *domain.Domain, db, tt, pp string, topn, buckets int) { + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(tt)) + require.NoError(t, err) tblInfo := tbl.Meta() physicalID := tblInfo.ID - if p != "global" { + if pp != "global" { for _, def := range tbl.Meta().GetPartitionInfo().Definitions { - if def.Name.L == p { + if def.Name.L == pp { physicalID = def.ID } } } - tblStats, err := s.do.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) - c.Assert(err, IsNil) + tblStats, err := dom.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) + require.NoError(t, err) delta := buckets/2 + 10 for _, idxStats := range tblStats.Indices { @@ -933,9 +909,9 @@ func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.Tes numBuckets := len(idxStats.Buckets) // since the hist-building algorithm doesn't stipulate the final bucket number to be equal to the expected number exactly, // we have to check the results by a range here. - c.Assert(numTopN, Equals, topn) - c.Assert(numBuckets, GreaterEqual, buckets-delta) - c.Assert(numBuckets, LessEqual, buckets+delta) + require.Equal(t, topn, numTopN) + require.GreaterOrEqual(t, numBuckets, buckets-delta) + require.LessOrEqual(t, numBuckets, buckets+delta) } for _, colStats := range tblStats.Columns { if len(colStats.Buckets) == 0 { @@ -943,19 +919,21 @@ func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.Tes } numTopN := colStats.TopN.Num() numBuckets := len(colStats.Buckets) - c.Assert(numTopN, Equals, topn) - c.Assert(numBuckets, GreaterEqual, buckets-delta) - c.Assert(numBuckets, LessEqual, buckets+delta) + require.Equal(t, topn, numTopN) + require.GreaterOrEqual(t, numBuckets, buckets-delta) + require.LessOrEqual(t, numBuckets, buckets+delta) } } -func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts1(c *C) { +func TestAnalyzeGlobalStatsWithOpts1(t *testing.T) { + t.Skip("unstable test") if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") + t.Skip("exhaustive types test, skip race test") } - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt") + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + prepareForGlobalStatsWithOpts(t, dom, tk, "test_gstats_opt", "test_gstats_opt") // nolint:unused type opt struct { @@ -978,54 +956,57 @@ func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts1(c *C) { sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) if !ca.err { tk.MustExec(sql) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) } else { err := tk.ExecToErr(sql) - c.Assert(err, NotNil) + require.Error(t, err) } } } -func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { +func TestAnalyzeGlobalStatsWithOpts2(t *testing.T) { + t.Skip("unstable test") if israce.RaceEnabled { - c.Skip("exhaustive types test, skip race test") + t.Skip("exhaustive types test, skip race test") } - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) originalVal1 := tk.MustQuery("select @@tidb_persist_analyze_options").Rows()[0][0].(string) defer func() { tk.MustExec(fmt.Sprintf("set global tidb_persist_analyze_options = %v", originalVal1)) }() tk.MustExec("set global tidb_persist_analyze_options=false") - s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2") + prepareForGlobalStatsWithOpts(t, dom, tk, "test_gstats_opt2", "test_gstats_opt2") tk.MustExec("analyze table test_gstats_opt2 with 20 topn, 50 buckets, 1000 samples") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 1, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 50) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 1, 50) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) // analyze a partition to let its options be different with others' tk.MustExec("analyze table test_gstats_opt2 partition p0 with 10 topn, 20 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 10, 20) // use new options - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 10, 20) // use new options + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) tk.MustExec("analyze table test_gstats_opt2 partition p1 with 100 topn, 200 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 100, 200) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 100, 200) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) tk.MustExec("analyze table test_gstats_opt2 partition p0 with 20 topn") // change back to 20 topn - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "global", 20, 256) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p0", 20, 256) + checkForGlobalStatsWithOpts(t, dom, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) } -func (s *testStatsSuite) TestGlobalStatsHealthy(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsHealthy(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(` @@ -1040,12 +1021,12 @@ partition by range (a) ( checkModifyAndCount := func(gModify, gCount, p0Modify, p0Count, p1Modify, p1Count int) { rs := tk.MustQuery("show stats_meta").Rows() - c.Assert(rs[0][4].(string), Equals, fmt.Sprintf("%v", gModify)) // global.modify_count - c.Assert(rs[0][5].(string), Equals, fmt.Sprintf("%v", gCount)) // global.row_count - c.Assert(rs[1][4].(string), Equals, fmt.Sprintf("%v", p0Modify)) // p0.modify_count - c.Assert(rs[1][5].(string), Equals, fmt.Sprintf("%v", p0Count)) // p0.row_count - c.Assert(rs[2][4].(string), Equals, fmt.Sprintf("%v", p1Modify)) // p1.modify_count - c.Assert(rs[2][5].(string), Equals, fmt.Sprintf("%v", p1Count)) // p1.row_count + require.Equal(t, fmt.Sprintf("%v", gModify), rs[0][4].(string)) // global.modify_count + require.Equal(t, fmt.Sprintf("%v", gCount), rs[0][5].(string)) // global.row_count + require.Equal(t, fmt.Sprintf("%v", p0Modify), rs[1][4].(string)) // p0.modify_count + require.Equal(t, fmt.Sprintf("%v", p0Count), rs[1][5].(string)) // p0.row_count + require.Equal(t, fmt.Sprintf("%v", p1Modify), rs[2][4].(string)) // p1.modify_count + require.Equal(t, fmt.Sprintf("%v", p1Count), rs[2][5].(string)) // p1.row_count } checkHealthy := func(gH, p0H, p1H int) { tk.MustQuery("show stats_healthy").Check(testkit.Rows( @@ -1061,14 +1042,14 @@ partition by range (a) ( checkHealthy(100, 100, 100) tk.MustExec("insert into t values (1), (2)") // update p0 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) checkModifyAndCount(2, 2, 2, 2, 0, 0) checkHealthy(0, 0, 100) tk.MustExec("insert into t values (11), (12), (13), (14)") // update p1 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) checkModifyAndCount(6, 6, 2, 2, 4, 4) checkHealthy(0, 0, 0) @@ -1077,26 +1058,28 @@ partition by range (a) ( checkHealthy(100, 100, 100) tk.MustExec("insert into t values (4), (5), (15), (16)") // update p0 and p1 together - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) checkModifyAndCount(4, 10, 2, 4, 2, 6) checkHealthy(60, 50, 66) } -func (s *testStatsSuite) TestHideGlobalStatsSwitch(c *C) { +func TestHideGlobalStatsSwitch(t *testing.T) { // NOTICE: remove this test when this global-stats is GA. - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) rs := tk.MustQuery("show variables").Rows() for _, r := range rs { - c.Assert(strings.ToLower(r[0].(string)), Not(Equals), "tidb_partition_prune_mode") + require.NotEqual(t, "tidb_partition_prune_mode", strings.ToLower(r[0].(string))) } - c.Assert(len(tk.MustQuery("show variables where variable_name like '%tidb_partition_prune_mode%'").Rows()), Equals, 0) + require.Len(t, tk.MustQuery("show variables where variable_name like '%tidb_partition_prune_mode%'").Rows(), 0) } -func (s *testStatsSuite) TestGlobalStatsData(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsData(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(` @@ -1111,7 +1094,7 @@ partition by range (a) ( tk.MustExec("set @@tidb_analyze_version=2") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (6), (null), (11), (12), (13), (14), (15), (16), (17), (18), (19), (19)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table t with 0 topn, 2 buckets") tk.MustQuery("select modify_count, count from mysql.stats_meta order by table_id asc").Check( @@ -1140,9 +1123,10 @@ partition by range (a) ( "test t p1 a 1 1 10 2 17 19 0")) } -func (s *testStatsSuite) TestGlobalStatsData2(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsData2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("set @@tidb_analyze_version=2") @@ -1151,7 +1135,7 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustExec("drop table if exists tint") tk.MustExec("create table tint (c int, key(c)) partition by range (c) (partition p0 values less than (10), partition p1 values less than (20))") tk.MustExec("insert into tint values (1), (2), (3), (4), (4), (5), (5), (5), (null), (11), (12), (13), (14), (15), (16), (16), (16), (16), (17), (17)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table tint with 2 topn, 2 buckets") tk.MustQuery("select modify_count, count from mysql.stats_meta order by table_id asc").Check(testkit.Rows( @@ -1210,13 +1194,13 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustExec(`insert into tdouble values ` + `(1, 1), (2, 2), (3, 3), (4, 4), (4, 4), (5, 5), (5, 5), (5, 5), (null, null), ` + // values in p0 `(11, 11), (12, 12), (13, 13), (14, 14), (15, 15), (16, 16), (16, 16), (16, 16), (16, 16), (17, 17), (17, 17)`) // values in p1 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table tdouble with 2 topn, 2 buckets") rs := tk.MustQuery("show stats_meta where table_name='tdouble'").Rows() - c.Assert(rs[0][5].(string), Equals, "20") // g.count = p0.count + p1.count - c.Assert(rs[1][5].(string), Equals, "9") // p0.count - c.Assert(rs[2][5].(string), Equals, "11") // p1.count + require.Equal(t, "20", rs[0][5].(string)) // g.count = p0.count + p1.count + require.Equal(t, "9", rs[1][5].(string)) // p0.count + require.Equal(t, "11", rs[2][5].(string)) // p1.count tk.MustQuery("show stats_topn where table_name='tdouble' and is_index=0 and column_name='c'").Check(testkit.Rows( `test tdouble global c 0 5 3`, @@ -1244,12 +1228,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdouble p1 c 0 1 5 1 14 15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdouble' and column_name='c' and is_index=0").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) tk.MustQuery("show stats_buckets where table_name='tdouble' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv @@ -1261,12 +1245,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdouble p1 c 1 1 5 1 14 15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdouble' and column_name='c' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) // decimal + (column + index with 1 column) tk.MustExec("drop table if exists tdecimal") @@ -1275,13 +1259,13 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustExec(`insert into tdecimal values ` + `(1, 1), (2, 2), (3, 3), (4, 4), (4, 4), (5, 5), (5, 5), (5, 5), (null, null), ` + // values in p0 `(11, 11), (12, 12), (13, 13), (14, 14), (15, 15), (16, 16), (16, 16), (16, 16), (16, 16), (17, 17), (17, 17)`) // values in p1 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table tdecimal with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tdecimal'").Rows() - c.Assert(rs[0][5].(string), Equals, "20") // g.count = p0.count + p1.count - c.Assert(rs[1][5].(string), Equals, "9") // p0.count - c.Assert(rs[2][5].(string), Equals, "11") // p1.count + require.Equal(t, "20", rs[0][5].(string)) // g.count = p0.count + p1.count + require.Equal(t, "9", rs[1][5].(string)) // p0.count + require.Equal(t, "11", rs[2][5].(string)) // p1.count tk.MustQuery("show stats_topn where table_name='tdecimal' and is_index=0 and column_name='c'").Check(testkit.Rows( `test tdecimal global c 0 5.00 3`, @@ -1309,12 +1293,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdecimal p1 c 0 1 5 1 14.00 15.00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdecimal' and column_name='c' and is_index=0").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) tk.MustQuery("show stats_buckets where table_name='tdecimal' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv @@ -1326,12 +1310,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdecimal p1 c 1 1 5 1 14.00 15.00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdecimal' and column_name='c' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) // datetime + (column + index with 1 column) tk.MustExec("drop table if exists tdatetime") @@ -1340,13 +1324,13 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustExec(`insert into tdatetime values ` + `(1, '2000-01-01'), (2, '2000-01-02'), (3, '2000-01-03'), (4, '2000-01-04'), (4, '2000-01-04'), (5, '2000-01-05'), (5, '2000-01-05'), (5, '2000-01-05'), (null, null), ` + // values in p0 `(11, '2000-01-11'), (12, '2000-01-12'), (13, '2000-01-13'), (14, '2000-01-14'), (15, '2000-01-15'), (16, '2000-01-16'), (16, '2000-01-16'), (16, '2000-01-16'), (16, '2000-01-16'), (17, '2000-01-17'), (17, '2000-01-17')`) // values in p1 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table tdatetime with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tdatetime'").Rows() - c.Assert(rs[0][5].(string), Equals, "20") // g.count = p0.count + p1.count - c.Assert(rs[1][5].(string), Equals, "9") // p0.count - c.Assert(rs[2][5].(string), Equals, "11") // p1.count + require.Equal(t, "20", rs[0][5].(string)) // g.count = p0.count + p1.count + require.Equal(t, "9", rs[1][5].(string)) // p0.count + require.Equal(t, "11", rs[2][5].(string)) // p1.count tk.MustQuery("show stats_topn where table_name='tdatetime' and is_index=0 and column_name='c'").Check(testkit.Rows( `test tdatetime global c 0 2000-01-05 00:00:00 3`, @@ -1374,12 +1358,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdatetime p1 c 0 1 5 1 2000-01-14 00:00:00 2000-01-15 00:00:00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdatetime' and column_name='c' and is_index=0").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) tk.MustQuery("show stats_buckets where table_name='tdatetime' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv @@ -1391,12 +1375,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tdatetime p1 c 1 1 5 1 2000-01-14 00:00:00 2000-01-15 00:00:00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdatetime' and column_name='c' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) // string + (column + index with 1 column) tk.MustExec("drop table if exists tstring") @@ -1405,13 +1389,13 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustExec(`insert into tstring values ` + `(1, 'a1'), (2, 'a2'), (3, 'a3'), (4, 'a4'), (4, 'a4'), (5, 'a5'), (5, 'a5'), (5, 'a5'), (null, null), ` + // values in p0 `(11, 'b11'), (12, 'b12'), (13, 'b13'), (14, 'b14'), (15, 'b15'), (16, 'b16'), (16, 'b16'), (16, 'b16'), (16, 'b16'), (17, 'b17'), (17, 'b17')`) // values in p1 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table tstring with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tstring'").Rows() - c.Assert(rs[0][5].(string), Equals, "20") // g.count = p0.count + p1.count - c.Assert(rs[1][5].(string), Equals, "9") // p0.count - c.Assert(rs[2][5].(string), Equals, "11") // p1.count + require.Equal(t, "20", rs[0][5].(string)) // g.count = p0.count + p1.count + require.Equal(t, "9", rs[1][5].(string)) // p0.count + require.Equal(t, "11", rs[2][5].(string)) // p1.count tk.MustQuery("show stats_topn where table_name='tstring' and is_index=0 and column_name='c'").Check(testkit.Rows( `test tstring global c 0 a5 3`, @@ -1439,12 +1423,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tstring p1 c 0 1 5 1 b14 b15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tstring' and column_name='c' and is_index=0").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) tk.MustQuery("show stats_buckets where table_name='tstring' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv @@ -1456,17 +1440,18 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { "test tstring p1 c 1 1 5 1 b14 b15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tstring' and column_name='c' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 - c.Assert(rs[1][6].(string), Equals, "5") - c.Assert(rs[2][6].(string), Equals, "7") - c.Assert(rs[0][7].(string), Equals, "1") // g.null_count = p0 + p1 - c.Assert(rs[1][7].(string), Equals, "1") - c.Assert(rs[2][7].(string), Equals, "0") -} - -func (s *testStatsSuite) TestGlobalStatsData3(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + require.Equal(t, "12", rs[0][6].(string)) // g.ndv = p0 + p1 + require.Equal(t, "5", rs[1][6].(string)) + require.Equal(t, "7", rs[2][6].(string)) + require.Equal(t, "1", rs[0][7].(string)) // g.null_count = p0 + p1 + require.Equal(t, "1", rs[1][7].(string)) + require.Equal(t, "0", rs[2][7].(string)) +} + +func TestGlobalStatsData3(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("set @@tidb_analyze_version=2") @@ -1480,9 +1465,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { tk.MustExec("analyze table tintint with 2 topn, 2 buckets") rs := tk.MustQuery("show stats_meta where table_name='tintint'").Rows() - c.Assert(rs[0][5].(string), Equals, "17") // g.total = p0.total + p1.total - c.Assert(rs[1][5].(string), Equals, "9") - c.Assert(rs[2][5].(string), Equals, "8") + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) tk.MustQuery("show stats_topn where table_name='tintint' and is_index=1").Check(testkit.Rows( "test tintint global a 1 (3, 1) 3", @@ -1501,9 +1486,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintint p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintint' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv - c.Assert(rs[1][6].(string), Equals, "6") - c.Assert(rs[2][6].(string), Equals, "5") + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) // index(int, string) tk.MustExec("drop table if exists tintstr") @@ -1514,9 +1499,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { tk.MustExec("analyze table tintstr with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tintstr'").Rows() - c.Assert(rs[0][5].(string), Equals, "17") // g.total = p0.total + p1.total - c.Assert(rs[1][5].(string), Equals, "9") - c.Assert(rs[2][5].(string), Equals, "8") + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) tk.MustQuery("show stats_topn where table_name='tintstr' and is_index=1").Check(testkit.Rows( "test tintstr global a 1 (3, 1) 3", @@ -1535,9 +1520,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintstr p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintstr' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv - c.Assert(rs[1][6].(string), Equals, "6") - c.Assert(rs[2][6].(string), Equals, "5") + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) // index(int, double) tk.MustExec("drop table if exists tintdouble") @@ -1548,9 +1533,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { tk.MustExec("analyze table tintdouble with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tintdouble'").Rows() - c.Assert(rs[0][5].(string), Equals, "17") // g.total = p0.total + p1.total - c.Assert(rs[1][5].(string), Equals, "9") - c.Assert(rs[2][5].(string), Equals, "8") + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) tk.MustQuery("show stats_topn where table_name='tintdouble' and is_index=1").Check(testkit.Rows( "test tintdouble global a 1 (3, 1) 3", @@ -1569,9 +1554,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintdouble p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintdouble' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv - c.Assert(rs[1][6].(string), Equals, "6") - c.Assert(rs[2][6].(string), Equals, "5") + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) // index(double, decimal) tk.MustExec("drop table if exists tdoubledecimal") @@ -1582,9 +1567,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { tk.MustExec("analyze table tdoubledecimal with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tdoubledecimal'").Rows() - c.Assert(rs[0][5].(string), Equals, "17") // g.total = p0.total + p1.total - c.Assert(rs[1][5].(string), Equals, "9") - c.Assert(rs[2][5].(string), Equals, "8") + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) tk.MustQuery("show stats_topn where table_name='tdoubledecimal' and is_index=1").Check(testkit.Rows( "test tdoubledecimal global a 1 (3, 1.00) 3", @@ -1603,9 +1588,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tdoubledecimal p1 a 1 1 3 1 (12, 2.00) (12, 2.00) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdoubledecimal' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv - c.Assert(rs[1][6].(string), Equals, "6") - c.Assert(rs[2][6].(string), Equals, "5") + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) // index(string, datetime) tk.MustExec("drop table if exists tstrdt") @@ -1616,9 +1601,9 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { tk.MustExec("analyze table tstrdt with 2 topn, 2 buckets") rs = tk.MustQuery("show stats_meta where table_name='tstrdt'").Rows() - c.Assert(rs[0][5].(string), Equals, "17") // g.total = p0.total + p1.total - c.Assert(rs[1][5].(string), Equals, "9") - c.Assert(rs[2][5].(string), Equals, "8") + require.Equal(t, "17", rs[0][5].(string)) // g.total = p0.total + p1.total + require.Equal(t, "9", rs[1][5].(string)) + require.Equal(t, "8", rs[2][5].(string)) tk.MustQuery("show stats_topn where table_name='tstrdt' and is_index=1").Check(testkit.Rows( "test tstrdt global a 1 (3, 2000-01-01 00:00:00) 3", @@ -1637,14 +1622,15 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tstrdt p1 a 1 1 3 1 (12, 2000-01-02 00:00:00) (12, 2000-01-02 00:00:00) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tstrdt' and is_index=1").Rows() - c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv - c.Assert(rs[1][6].(string), Equals, "6") - c.Assert(rs[2][6].(string), Equals, "5") + require.Equal(t, "11", rs[0][6].(string)) // g.ndv = p0.ndv + p1.ndv + require.Equal(t, "6", rs[1][6].(string)) + require.Equal(t, "5", rs[2][6].(string)) } -func (s *testStatsSuite) TestGlobalStatsVersion(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsVersion(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec(` @@ -1656,62 +1642,63 @@ partition by range (a) ( partition p1 values less than (20) )`) tk.MustExec("insert into t values (1), (5), (null), (11), (15)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("set @@session.tidb_analyze_version=1") tk.MustExec("analyze table t") // both p0 and p1 are in ver1 - c.Assert(len(tk.MustQuery("show stats_meta").Rows()), Equals, 2) + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 2) tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("set @@session.tidb_analyze_version=1") err := tk.ExecToErr("analyze table t") // try to build global-stats on ver1 - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") tk.MustExec("set @@session.tidb_analyze_version=2") err = tk.ExecToErr("analyze table t partition p1") // only analyze p1 to let it in ver2 while p0 is in ver1 - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("analyze table t") // both p0 and p1 are in ver2 - c.Assert(len(tk.MustQuery("show stats_meta").Rows()), Equals, 3) + require.Len(t, tk.MustQuery("show stats_meta").Rows(), 3) // If we already have global-stats, we can get the latest global-stats by analyzing the newly added partition. tk.MustExec("alter table t add partition (partition p2 values less than (30))") tk.MustExec("insert t values (13), (14), (22), (23)") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table t partition p2") // it will success since p0 and p1 are both in ver2 - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - do := s.do + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + do := dom is := do.InfoSchema() h := do.StatsHandle() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() globalStats := h.GetTableStats(tableInfo) // global.count = p0.count(3) + p1.count(2) + p2.count(2) // We did not analyze partition p1, so the value here has not changed - c.Assert(globalStats.Count, Equals, int64(7)) + require.Equal(t, int64(7), globalStats.Count) tk.MustExec("analyze table t partition p1;") globalStats = h.GetTableStats(tableInfo) // global.count = p0.count(3) + p1.count(4) + p2.count(4) // The value of p1.Count is correct now. - c.Assert(globalStats.Count, Equals, int64(9)) - c.Assert(globalStats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(9), globalStats.Count) + require.Equal(t, int64(0), globalStats.ModifyCount) tk.MustExec("alter table t drop partition p2;") - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) tk.MustExec("analyze table t;") globalStats = h.GetTableStats(tableInfo) // global.count = p0.count(3) + p1.count(4) - c.Assert(globalStats.Count, Equals, int64(7)) + require.Equal(t, int64(7), globalStats.Count) } -func (s *testSerialStatsSuite) TestDDLPartition4GlobalStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDDLPartition4GlobalStats(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("set @@session.tidb_analyze_version=2") @@ -1724,57 +1711,58 @@ func (s *testSerialStatsSuite) TestDDLPartition4GlobalStats(c *C) { partition p4 values less than (50), partition p5 values less than (60) )`) - do := s.do + do := dom is := do.InfoSchema() h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) tk.MustExec("insert into t values (1), (2), (3), (4), (5), " + "(11), (21), (31), (41), (51)," + "(12), (22), (32), (42), (52);") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) tk.MustExec("analyze table t") result := tk.MustQuery("show stats_meta where table_name = 't';").Rows() - c.Assert(len(result), Equals, 7) + require.Len(t, result, 7) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() globalStats := h.GetTableStats(tableInfo) - c.Assert(globalStats.Count, Equals, int64(15)) + require.Equal(t, int64(15), globalStats.Count) tk.MustExec("alter table t drop partition p3, p5;") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.Update(is)) result = tk.MustQuery("show stats_meta where table_name = 't';").Rows() - c.Assert(len(result), Equals, 5) + require.Len(t, result, 5) // The value of global.count will be updated automatically after we drop the table partition. globalStats = h.GetTableStats(tableInfo) - c.Assert(globalStats.Count, Equals, int64(11)) + require.Equal(t, int64(11), globalStats.Count) tk.MustExec("alter table t truncate partition p2, p4;") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) + require.NoError(t, h.Update(is)) // The value of global.count will not be updated automatically when we truncate the table partition. // Because the partition-stats in the partition table which have been truncated has not been updated. globalStats = h.GetTableStats(tableInfo) - c.Assert(globalStats.Count, Equals, int64(11)) + require.Equal(t, int64(11), globalStats.Count) tk.MustExec("analyze table t;") result = tk.MustQuery("show stats_meta where table_name = 't';").Rows() // The truncate operation only delete the data from the partition p2 and p4. It will not delete the partition-stats. - c.Assert(len(result), Equals, 5) + require.Len(t, result, 5) // The result for the globalStats.count will be right now globalStats = h.GetTableStats(tableInfo) - c.Assert(globalStats.Count, Equals, int64(7)) + require.Equal(t, int64(7), globalStats.Count) } -func (s *testStatsSuite) TestMergeGlobalTopN(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestMergeGlobalTopN(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("drop table if exists t;") tk.MustExec("set @@session.tidb_analyze_version=2;") @@ -1808,81 +1796,84 @@ func (s *testStatsSuite) TestMergeGlobalTopN(c *C) { ("test t global b 1 3 5"))) } -func (s *testStatsSuite) TestExtendedStatsDefaultSwitch(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtendedStatsDefaultSwitch(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int primary key, b int, c int, d int)") err := tk.ExecToErr("alter table t add stats_extended s1 correlation(b,c)") - c.Assert(err.Error(), Equals, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF") + require.Equal(t, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF", err.Error()) err = tk.ExecToErr("alter table t drop stats_extended s1") - c.Assert(err.Error(), Equals, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF") + require.Equal(t, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF", err.Error()) err = tk.ExecToErr("admin reload stats_extended") - c.Assert(err.Error(), Equals, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF") + require.Equal(t, "Extended statistics feature is not generally available now, and tidb_enable_extended_stats is OFF", err.Error()) } -func (s *testStatsSuite) TestExtendedStatsOps(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtendedStatsOps(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int primary key, b int, c int, d int)") tk.MustExec("insert into t values(1,1,5,1),(2,2,4,2),(3,3,3,3),(4,4,2,4),(5,5,1,5)") tk.MustExec("analyze table t") err := tk.ExecToErr("alter table not_exist_db.t add stats_extended s1 correlation(b,c)") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'not_exist_db.t' doesn't exist") + require.Equal(t, "[schema:1146]Table 'not_exist_db.t' doesn't exist", err.Error()) err = tk.ExecToErr("alter table not_exist_tbl add stats_extended s1 correlation(b,c)") - c.Assert(err.Error(), Equals, "[schema:1146]Table 'test.not_exist_tbl' doesn't exist") + require.Equal(t, "[schema:1146]Table 'test.not_exist_tbl' doesn't exist", err.Error()) err = tk.ExecToErr("alter table t add stats_extended s1 correlation(b,e)") - c.Assert(err.Error(), Equals, "[schema:1054]Unknown column 'e' in 't'") + require.Equal(t, "[schema:1054]Unknown column 'e' in 't'", err.Error()) tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustQuery("show warnings").Check(testkit.Rows( "Warning 1105 No need to create correlation statistics on the integer primary key column", )) tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) err = tk.ExecToErr("alter table t add stats_extended s1 correlation(b,c,d)") - c.Assert(err.Error(), Equals, "Only support Correlation and Dependency statistics types on 2 columns") + require.Equal(t, "Only support Correlation and Dependency statistics types on 2 columns", err.Error()) tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) tk.MustExec("alter table t add stats_extended s1 correlation(b,c)") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( "2 [2,3] 0", )) - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) tk.MustExec("alter table t drop stats_extended s1") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( "2 [2,3] 2", )) err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) } -func (s *testStatsSuite) TestAdminReloadStatistics1(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAdminReloadStatistics1(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int primary key, b int, c int, d int)") @@ -1892,43 +1883,44 @@ func (s *testStatsSuite) TestAdminReloadStatistics1(c *C) { tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows( "2 [2,3] 0", )) - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) tk.MustExec("update mysql.stats_extended set status = 1 where name = 's1'") do.StatsHandle().Clear() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) tk.MustExec("delete from mysql.stats_extended where name = 's1'") err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) tk.MustExec("admin reload stats_extended") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) } -func (s *testStatsSuite) TestAdminReloadStatistics2(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAdminReloadStatistics2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -1939,24 +1931,25 @@ func (s *testStatsSuite) TestAdminReloadStatistics2(c *C) { "1.000000 1", )) rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) tk.MustExec("delete from mysql.stats_extended where name = 's1'") - is := s.do.InfoSchema() - s.do.StatsHandle().Update(is) + is := dom.InfoSchema() + dom.StatsHandle().Update(is) tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) tk.MustExec("admin reload stats_extended") tk.MustQuery("select stats, status from mysql.stats_extended where name = 's1'").Check(testkit.Rows()) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestCorrelationStatsCompute(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int)") @@ -1969,17 +1962,17 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,2] 0", "2 [1,3] 0", )) - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( @@ -1993,25 +1986,25 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,3] -1.000000 1", )) err = do.StatsHandle().Update(is) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 2) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 2) foundS1, foundS2 := false, false for name, item := range statsTbl.ExtendedStats.Stats { switch name { case "s1": foundS1 = true - c.Assert(item.ScalarVals, Equals, float64(1)) + require.Equal(t, float64(1), item.ScalarVals) case "s2": foundS2 = true - c.Assert(item.ScalarVals, Equals, float64(-1)) + require.Equal(t, float64(-1), item.ScalarVals) default: - c.Assert("Unexpected extended stats in cache", IsNil) + require.FailNow(t, "Unexpected extended stats in cache") } } - c.Assert(foundS1 && foundS2, IsTrue) + require.True(t, foundS1 && foundS2) // Check that table with NULLs won't cause panic tk.MustExec("delete from t") @@ -2043,41 +2036,43 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { )) } -func (s *testStatsSuite) TestSyncStatsExtendedRemoval(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestSyncStatsExtendedRemoval(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values(1,1),(2,2),(3,3)") tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 1) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 1) item := statsTbl.ExtendedStats.Stats["s1"] - c.Assert(item, NotNil) + require.NotNil(t, item) result := tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) tk.MustExec("alter table t drop stats_extended s1") statsTbl = do.StatsHandle().GetTableStats(tableInfo) - c.Assert(statsTbl, NotNil) - c.Assert(statsTbl.ExtendedStats, NotNil) - c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + require.NotNil(t, statsTbl) + require.NotNil(t, statsTbl.ExtendedStats) + require.Len(t, statsTbl.ExtendedStats.Stats, 0) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) } -func (s *testStatsSuite) TestStaticPartitionPruneMode(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestStaticPartitionPruneMode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") tk.MustExec("use test") tk.MustExec(`create table t (a int, key(a)) partition by range(a) @@ -2085,22 +2080,23 @@ func (s *testStatsSuite) TestStaticPartitionPruneMode(c *C) { partition p1 values less than (22))`) tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) tk.MustExec(`analyze table t`) - c.Assert(tk.MustNoGlobalStats("t"), IsTrue) + require.True(t, tk.MustNoGlobalStats("t")) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") - c.Assert(tk.MustNoGlobalStats("t"), IsTrue) + require.True(t, tk.MustNoGlobalStats("t")) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") tk.MustExec(`insert into t values (4), (5), (6)`) tk.MustExec(`analyze table t partition p0`) - c.Assert(tk.MustNoGlobalStats("t"), IsTrue) + require.True(t, tk.MustNoGlobalStats("t")) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") - c.Assert(tk.MustNoGlobalStats("t"), IsTrue) + require.True(t, tk.MustNoGlobalStats("t")) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") } -func (s *testStatsSuite) TestMergeIdxHist(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestMergeIdxHist(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Dynamic) + "'") defer tk.MustExec("set @@tidb_partition_prune_mode='" + string(variable.Static) + "'") tk.MustExec("use test") @@ -2115,12 +2111,13 @@ func (s *testStatsSuite) TestMergeIdxHist(c *C) { tk.MustExec("analyze table t with 2 topn, 2 buckets") rows := tk.MustQuery("show stats_buckets where partition_name like 'global'") - c.Assert(len(rows.Rows()), Equals, 4) + require.Len(t, rows.Rows(), 4) } -func (s *testStatsSuite) TestAnalyzeWithDynamicPartitionPruneMode(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAnalyzeWithDynamicPartitionPruneMode(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") tk.MustExec("set @@tidb_analyze_version = 2") @@ -2130,28 +2127,29 @@ func (s *testStatsSuite) TestAnalyzeWithDynamicPartitionPruneMode(c *C) { tk.MustExec(`insert into t values (1), (2), (3), (10), (11)`) tk.MustExec(`analyze table t with 1 topn, 2 buckets`) rows := tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[1][6], Equals, "4") + require.Len(t, rows, 2) + require.Equal(t, "4", rows[1][6]) tk.MustExec("insert into t values (1), (2), (2)") tk.MustExec("analyze table t partition p0 with 1 topn, 2 buckets") rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[1][6], Equals, "5") + require.Len(t, rows, 2) + require.Equal(t, "5", rows[1][6]) tk.MustExec("insert into t values (3)") tk.MustExec("analyze table t partition p0 index a with 1 topn, 2 buckets") rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][6], Equals, "6") + require.Len(t, rows, 1) + require.Equal(t, "6", rows[0][6]) } -func (s *testStatsSuite) TestPartitionPruneModeSessionVariable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk1 := testkit.NewTestKit(c, s.store) +func TestPartitionPruneModeSessionVariable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk1 := testkit.NewTestKit(t, store) tk1.MustExec("use test") tk1.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") tk1.MustExec(`set @@tidb_analyze_version=2`) - tk2 := testkit.NewTestKit(c, s.store) + tk2 := testkit.NewTestKit(t, store) tk2.MustExec("use test") tk2.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Static) + "'") tk2.MustExec(`set @@tidb_analyze_version=2`) @@ -2201,9 +2199,10 @@ func (s *testStatsSuite) TestPartitionPruneModeSessionVariable(c *C) { )) } -func (s *testStatsSuite) TestFMSWithAnalyzePartition(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestFMSWithAnalyzePartition(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode = '" + string(variable.Dynamic) + "'") tk.MustExec("set @@tidb_analyze_version = 2") @@ -2221,17 +2220,12 @@ func (s *testStatsSuite) TestFMSWithAnalyzePartition(c *C) { tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) } -var _ = SerialSuites(&statsSerialSuite{}) - -type statsSerialSuite struct { - testSuiteBase -} - -func (s *statsSerialSuite) TestIndexUsageInformation(c *C) { - defer cleanEnv(c, s.store, s.do) +func TestIndexUsageInformation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() session.SetIndexUsageSyncLease(1) defer session.SetIndexUsageSyncLease(0) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_idx(a int, b int)") tk.MustExec("create unique index idx_a on t_idx(a)") @@ -2244,9 +2238,9 @@ func (s *statsSerialSuite) TestIndexUsageInformation(c *C) { AND tables.tidb_table_id = stats.table_id AND idx.index_id = stats.index_id AND idx.table_name = "t_idx"` - do := s.do + do := dom err := do.StatsHandle().DumpIndexUsageToKV() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery(querySQL).Check(testkit.Rows( "test t_idx idx_a 1 0", )) @@ -2254,32 +2248,33 @@ func (s *statsSerialSuite) TestIndexUsageInformation(c *C) { tk.MustQuery("select a from t_idx where a=1") tk.MustQuery("select a from t_idx where a=1") err = do.StatsHandle().DumpIndexUsageToKV() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery(querySQL).Check(testkit.Rows( "test t_idx idx_a 3 2", )) tk.MustQuery("select b from t_idx where b=0") tk.MustQuery("select b from t_idx where b=0") err = do.StatsHandle().DumpIndexUsageToKV() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery(querySQL).Check(testkit.Rows( "test t_idx idx_a 3 2", "test t_idx idx_b 2 2", )) } -func (s *statsSerialSuite) TestGCIndexUsageInformation(c *C) { - defer cleanEnv(c, s.store, s.do) +func TestGCIndexUsageInformation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() session.SetIndexUsageSyncLease(1) defer session.SetIndexUsageSyncLease(0) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t_idx(a int, b int)") tk.MustExec("create unique index idx_a on t_idx(a)") tk.MustQuery("select a from t_idx where a=1") - do := s.do + do := dom err := do.StatsHandle().DumpIndexUsageToKV() - c.Assert(err, IsNil) + require.NoError(t, err) querySQL := `select count(distinct idx.table_schema, idx.table_name, idx.key_name, stats.query_count, stats.rows_selected) from mysql.schema_index_usage as stats, information_schema.tidb_indexes as idx, information_schema.tables as tables where tables.table_schema = idx.table_schema @@ -2290,13 +2285,14 @@ func (s *statsSerialSuite) TestGCIndexUsageInformation(c *C) { tk.MustQuery(querySQL).Check(testkit.Rows("1")) tk.MustExec("drop index `idx_a` on t_idx") err = do.StatsHandle().GCIndexUsage() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustQuery(querySQL).Check(testkit.Rows("0")) } -func (s *statsSerialSuite) TestFeedbackWithGlobalStats(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestFeedbackWithGlobalStats(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@tidb_analyze_version = 1") @@ -2318,19 +2314,19 @@ func (s *statsSerialSuite) TestFeedbackWithGlobalStats(c *C) { testKit.MustQuery("show warnings").Check(testkit.Rows(`Error 1105 variable tidb_analyze_version not updated because analyze version 2 is incompatible with query feedback. Please consider setting feedback-probability to 0.0 in config file to disable query feedback`)) testKit.MustQuery("select @@tidb_analyze_version").Check(testkit.Rows("1")) - h := s.do.StatsHandle() + h := dom.StatsHandle() var err error // checkFeedbackOnPartitionTable is used to check whether the statistics are the same as before. checkFeedbackOnPartitionTable := func(statsBefore *statistics.Table, tblInfo *model.TableInfo) { - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) err = h.DumpStatsFeedbackToKV() - c.Assert(err, IsNil) - err = h.HandleUpdateStats(s.do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, err) + err = h.HandleUpdateStats(dom.InfoSchema()) + require.NoError(t, err) statsTblAfter := h.GetTableStats(tblInfo) // assert that statistics not changed // the feedback can not work for the partition table in both static and dynamic mode - assertTableEqual(c, statsBefore, statsTblAfter) + assertTableEqual(t, statsBefore, statsTblAfter) } // Case 2: Feedback wouldn't be applied on version 2 and global-level statistics. @@ -2343,13 +2339,13 @@ func (s *statsSerialSuite) TestFeedbackWithGlobalStats(c *C) { testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") } testKit.MustExec("analyze table t with 0 topn") - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() testKit.MustExec("analyze table t") - err = h.Update(s.do.InfoSchema()) - c.Assert(err, IsNil) + err = h.Update(dom.InfoSchema()) + require.NoError(t, err) statsTblBefore := h.GetTableStats(tblInfo) statistics.FeedbackProbability.Store(1) // make the statistics inaccurate. @@ -2375,9 +2371,9 @@ func (s *statsSerialSuite) TestFeedbackWithGlobalStats(c *C) { for i := 0; i < 200; i++ { testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") } - is = s.do.InfoSchema() + is = dom.InfoSchema() table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo = table.Meta() statsTblBefore = h.GetTableStats(tblInfo) // trigger feedback @@ -2388,44 +2384,48 @@ func (s *statsSerialSuite) TestFeedbackWithGlobalStats(c *C) { checkFeedbackOnPartitionTable(statsTblBefore, tblInfo) } -func (s *testStatsSuite) TestExtendedStatsPartitionTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtendedStatsPartitionTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int) partition by range(a) (partition p0 values less than (5), partition p1 values less than (10))") tk.MustExec("create table t2(a int, b int, c int) partition by hash(a) partitions 4") err := tk.ExecToErr("alter table t1 add stats_extended s1 correlation(b,c)") - c.Assert(err.Error(), Equals, "Extended statistics on partitioned tables are not supported now") + require.Equal(t, "Extended statistics on partitioned tables are not supported now", err.Error()) err = tk.ExecToErr("alter table t2 add stats_extended s1 correlation(b,c)") - c.Assert(err.Error(), Equals, "Extended statistics on partitioned tables are not supported now") + require.Equal(t, "Extended statistics on partitioned tables are not supported now", err.Error()) } -func (s *testStatsSuite) TestHideIndexUsageSyncLease(c *C) { +func TestHideIndexUsageSyncLease(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // NOTICE: remove this test when index usage is GA. - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) rs := tk.MustQuery("select @@tidb_config").Rows() for _, r := range rs { - c.Assert(strings.Contains(strings.ToLower(r[0].(string)), "index-usage-sync-lease"), IsFalse) + require.False(t, strings.Contains(strings.ToLower(r[0].(string)), "index-usage-sync-lease")) } } -func (s *testStatsSuite) TestHideExtendedStatsSwitch(c *C) { +func TestHideExtendedStatsSwitch(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() // NOTICE: remove this test when this extended-stats reaches GA state. - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + tk := testkit.NewTestKit(t, store) rs := tk.MustQuery("show variables").Rows() for _, r := range rs { - c.Assert(strings.ToLower(r[0].(string)), Not(Equals), "tidb_enable_extended_stats") + require.NotEqual(t, "tidb_enable_extended_stats", strings.ToLower(r[0].(string))) } tk.MustQuery("show variables like 'tidb_enable_extended_stats'").Check(testkit.Rows()) } -func (s *testStatsSuite) TestRepetitiveAddDropExtendedStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestRepetitiveAddDropExtendedStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -2435,58 +2435,60 @@ func (s *testStatsSuite) TestRepetitiveAddDropExtendedStats(c *C) { "s1 0", )) result := tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("analyze table t") tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( "s1 1", )) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) tk.MustExec("alter table t drop stats_extended s1") tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( "s1 2", )) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( "s1 0", )) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) tk.MustExec("analyze table t") tk.MustQuery("select name, status from mysql.stats_extended where name = 's1'").Sort().Check(testkit.Rows( "s1 1", )) result = tk.MustQuery("show stats_extended where db_name = 'test' and table_name = 't'") - c.Assert(len(result.Rows()), Equals, 1) + require.Len(t, result.Rows(), 1) } -func (s *testStatsSuite) TestDuplicateExtendedStats(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDuplicateExtendedStats(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int)") err := tk.ExecToErr("alter table t add stats_extended s1 correlation(a,a)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "Cannot create extended statistics on duplicate column names 'a'") + require.Error(t, err) + require.Equal(t, "Cannot create extended statistics on duplicate column names 'a'", err.Error()) tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") err = tk.ExecToErr("alter table t add stats_extended s1 correlation(a,c)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "extended statistics 's1' for the specified table already exists") + require.Error(t, err) + require.Equal(t, "extended statistics 's1' for the specified table already exists", err.Error()) err = tk.ExecToErr("alter table t add stats_extended s2 correlation(a,b)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "extended statistics 's2' with same type on same columns already exists") + require.Error(t, err) + require.Equal(t, "extended statistics 's2' with same type on same columns already exists", err.Error()) err = tk.ExecToErr("alter table t add stats_extended s2 correlation(b,a)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "extended statistics 's2' with same type on same columns already exists") + require.Error(t, err) + require.Equal(t, "extended statistics 's2' with same type on same columns already exists", err.Error()) tk.MustExec("alter table t add stats_extended s2 correlation(a,c)") } -func (s *testStatsSuite) TestDuplicateFMSketch(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDuplicateFMSketch(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") defer tk.MustExec("set @@tidb_partition_prune_mode='static'") @@ -2498,13 +2500,14 @@ func (s *testStatsSuite) TestDuplicateFMSketch(c *C) { tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) tk.MustExec("alter table t drop column b") - c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), time.Duration(0)), IsNil) + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), time.Duration(0))) tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) } -func (s *testStatsSuite) TestIndexFMSketch(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestIndexFMSketch(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 1") tk.MustExec("drop table if exists t") @@ -2519,7 +2522,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { tk.MustExec("analyze table t") tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("15")) tk.MustExec("drop table if exists t") - c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) // clustered index tk.MustExec("drop table if exists t") @@ -2529,17 +2532,17 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { tk.MustExec("analyze table t") tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) tk.MustExec("drop table if exists t") - c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) // test NDV checkNDV := func(rows, ndv int) { tk.MustExec("analyze table t") rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() - c.Assert(len(rs), Equals, rows) + require.Len(t, rs, rows) for i := range rs { fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) - c.Assert(err, IsNil) - c.Assert(fm.NDV(), Equals, int64(ndv)) + require.NoError(t, err) + require.Equal(t, int64(ndv), fm.NDV()) } } @@ -2552,7 +2555,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { tk.MustExec("insert into t values (2), (5)") checkNDV(6, 2) tk.MustExec("drop table if exists t") - c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) + require.NoError(t, dom.StatsHandle().GCStats(dom.InfoSchema(), 0)) // clustered index tk.MustExec("set @@tidb_enable_clustered_index=ON") @@ -2563,9 +2566,10 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { checkNDV(6, 2) } -func (s *testStatsSuite) TestShowExtendedStats4DropColumn(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestShowExtendedStats4DropColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int, c int)") @@ -2574,25 +2578,26 @@ func (s *testStatsSuite) TestShowExtendedStats4DropColumn(c *C) { tk.MustExec("alter table t add stats_extended s2 correlation(b,c)") tk.MustExec("analyze table t") rows := tk.MustQuery("show stats_extended").Sort().Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0][2], Equals, "s1") - c.Assert(rows[0][3], Equals, "[a,b]") - c.Assert(rows[1][2], Equals, "s2") - c.Assert(rows[1][3], Equals, "[b,c]") + require.Len(t, rows, 2) + require.Equal(t, "s1", rows[0][2]) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "s2", rows[1][2]) + require.Equal(t, "[b,c]", rows[1][3]) tk.MustExec("alter table t drop column b") rows = tk.MustQuery("show stats_extended").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) // Previously registered extended stats should be invalid for re-created columns. tk.MustExec("alter table t add column b int") rows = tk.MustQuery("show stats_extended").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testStatsSuite) TestGlobalStatsNDV(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsNDV(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") tk.MustExec(`CREATE TABLE t ( a int, key(a) ) @@ -2605,9 +2610,9 @@ func (s *testStatsSuite) TestGlobalStatsNDV(c *C) { checkNDV := func(ndvs ...int) { // g, p0, ..., p3 tk.MustExec("analyze table t") rs := tk.MustQuery(`show stats_histograms where is_index=1`).Rows() - c.Assert(len(rs), Equals, 5) + require.Len(t, rs, 5) for i, ndv := range ndvs { - c.Assert(rs[i][6].(string), Equals, fmt.Sprintf("%v", ndv)) + require.Equal(t, fmt.Sprintf("%v", ndv), rs[i][6].(string)) } } @@ -2632,19 +2637,20 @@ func (s *testStatsSuite) TestGlobalStatsNDV(c *C) { checkNDV(13, 3, 3, 3, 4) } -func (s *testStatsSuite) TestGlobalStatsIndexNDV(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestGlobalStatsIndexNDV(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") checkNDV := func(tbl string, g int, ps ...int) { // g, p0, ..., p3 tk.MustExec("analyze table " + tbl) rs := tk.MustQuery(fmt.Sprintf(`show stats_histograms where is_index=1 and table_name='%v'`, tbl)).Rows() - c.Assert(len(rs), Equals, 1+len(ps)) // 1(global) + number of partitions - c.Assert(rs[0][6].(string), Equals, fmt.Sprintf("%v", g)) // global + require.Len(t, rs, 1+len(ps)) // 1(global) + number of partitions + require.Equal(t, fmt.Sprintf("%v", g), rs[0][6].(string)) // global for i, ndv := range ps { - c.Assert(rs[i+1][6].(string), Equals, fmt.Sprintf("%v", ndv)) + require.Equal(t, fmt.Sprintf("%v", ndv), rs[i+1][6].(string)) } } @@ -2724,9 +2730,10 @@ func (s *testStatsSuite) TestGlobalStatsIndexNDV(c *C) { checkNDV("tdatetime", 8, 8, 6) } -func (s *testStatsSuite) TestExtStatsOnReCreatedTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtStatsOnReCreatedTable(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -2734,36 +2741,37 @@ func (s *testStatsSuite) TestExtStatsOnReCreatedTable(c *C) { tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") rows := tk.MustQuery("select table_id, stats from mysql.stats_extended where name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) + require.Len(t, rows, 1) tableID1 := rows[0][0] - c.Assert(rows[0][1], Equals, "1.000000") + require.Equal(t, "1.000000", rows[0][1]) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][5], Equals, "1.000000") + require.Len(t, rows, 1) + require.Equal(t, "1.000000", rows[0][5]) tk.MustExec("drop table t") rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values(1,3),(2,2),(3,1)") tk.MustExec("alter table t add stats_extended s1 correlation(a,b)") tk.MustExec("analyze table t") rows = tk.MustQuery("select table_id, stats from mysql.stats_extended where name = 's1' order by stats").Rows() - c.Assert(len(rows), Equals, 2) + require.Len(t, rows, 2) tableID2 := rows[0][0] - c.Assert(tableID2, Not(Equals), tableID1) - c.Assert(rows[1][0], Equals, tableID1) - c.Assert(rows[0][1], Equals, "-1.000000") - c.Assert(rows[1][1], Equals, "1.000000") + require.NotEqual(t, tableID1, tableID2) + require.Equal(t, tableID1, rows[1][0]) + require.Equal(t, "-1.000000", rows[0][1]) + require.Equal(t, "1.000000", rows[1][1]) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][5], Equals, "-1.000000") + require.Len(t, rows, 1) + require.Equal(t, "-1.000000", rows[0][5]) } -func (s *testStatsSuite) TestExtStatsOnReCreatedColumn(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtStatsOnReCreatedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -2774,9 +2782,9 @@ func (s *testStatsSuite) TestExtStatsOnReCreatedColumn(c *C) { "[1,2] 1.000000", )) rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[a,b]") - c.Assert(rows[0][5], Equals, "1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) tk.MustExec("alter table t drop column b") tk.MustExec("alter table t add column b int") @@ -2799,12 +2807,13 @@ func (s *testStatsSuite) TestExtStatsOnReCreatedColumn(c *C) { "[1,2] 1.000000", )) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 0) + require.Len(t, rows, 0) } -func (s *testStatsSuite) TestExtStatsOnRenamedColumn(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtStatsOnRenamedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -2815,9 +2824,9 @@ func (s *testStatsSuite) TestExtStatsOnRenamedColumn(c *C) { "[1,2] 1.000000", )) rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[a,b]") - c.Assert(rows[0][5], Equals, "1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) tk.MustExec("alter table t rename column b to c") tk.MustExec("update t set c = 3 where a = 1") @@ -2834,14 +2843,15 @@ func (s *testStatsSuite) TestExtStatsOnRenamedColumn(c *C) { "[1,2] -1.000000", )) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[a,c]") - c.Assert(rows[0][5], Equals, "-1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[a,c]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) } -func (s *testStatsSuite) TestExtStatsOnModifiedColumn(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestExtStatsOnModifiedColumn(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set session tidb_enable_extended_stats = on") tk.MustExec("use test") tk.MustExec("create table t(a int, b int)") @@ -2852,9 +2862,9 @@ func (s *testStatsSuite) TestExtStatsOnModifiedColumn(c *C) { "[1,2] 1.000000", )) rows := tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[a,b]") - c.Assert(rows[0][5], Equals, "1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "1.000000", rows[0][5]) tk.MustExec("alter table t modify column b bigint") tk.MustExec("update t set b = 3 where a = 1") @@ -2871,16 +2881,15 @@ func (s *testStatsSuite) TestExtStatsOnModifiedColumn(c *C) { "[1,2] -1.000000", )) rows = tk.MustQuery("show stats_extended where stats_name = 's1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[a,b]") - c.Assert(rows[0][5], Equals, "-1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[a,b]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) } -func (s *testSerialStatsSuite) TestCorrelationWithDefinedCollate(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) +func TestCorrelationWithDefinedCollate(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a int primary key, b varchar(8) character set utf8mb4 collate utf8mb4_general_ci, c varchar(8) character set utf8mb4 collate utf8mb4_bin)") @@ -2897,42 +2906,42 @@ func (s *testSerialStatsSuite) TestCorrelationWithDefinedCollate(c *C) { "1", )) rows := testKit.MustQuery("show stats_histograms where table_name = 't'").Sort().Rows() - c.Assert(len(rows), Equals, 3) - c.Assert(rows[1][9], Equals, "1") - c.Assert(rows[2][9], Equals, "-1") + require.Len(t, rows, 3) + require.Equal(t, "1", rows[1][9]) + require.Equal(t, "-1", rows[2][9]) testKit.MustExec("set session tidb_enable_extended_stats = on") testKit.MustExec("alter table t add stats_extended s1 correlation(b,c)") testKit.MustExec("analyze table t") rows = testKit.MustQuery("show stats_extended where stats_name = 's1'").Sort().Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "[b,c]") - c.Assert(rows[0][5], Equals, "-1.000000") + require.Len(t, rows, 1) + require.Equal(t, "[b,c]", rows[0][3]) + require.Equal(t, "-1.000000", rows[0][5]) } -func (s *testSerialStatsSuite) TestLoadHistogramWithCollate(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) +func TestLoadHistogramWithCollate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a varchar(10) collate utf8mb4_unicode_ci);") testKit.MustExec("insert into t values('abcdefghij');") testKit.MustExec("insert into t values('abcdufghij');") testKit.MustExec("analyze table t with 0 topn;") - do := s.do + do := dom h := do.StatsHandle() is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() _, err = h.TableStatsFromStorage(tblInfo, tblInfo.ID, true, 0) - c.Assert(err, IsNil) + require.NoError(t, err) } -func (s *testSerialStatsSuite) TestFastAnalyzeColumnHistWithNullValue(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestFastAnalyzeColumnHistWithNullValue(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t (a int)") @@ -2945,30 +2954,32 @@ func (s *testSerialStatsSuite) TestFastAnalyzeColumnHistWithNullValue(c *C) { testKit.MustQuery("select min(lower_bound) from mysql.stats_buckets").Check(testkit.Rows("1")) } -func (s *testStatsSuite) TestStatsCacheUpdateSkip(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - do := s.do +func TestStatsCacheUpdateSkip(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + do := dom h := do.StatsHandle() testKit.MustExec("use test") testKit.MustExec("create table t (c1 int, c2 int)") testKit.MustExec("insert into t values(1, 2)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() statsTbl1 := h.GetTableStats(tableInfo) - c.Assert(statsTbl1.Pseudo, IsFalse) + require.False(t, statsTbl1.Pseudo) h.Update(is) statsTbl2 := h.GetTableStats(tableInfo) - c.Assert(statsTbl1, Equals, statsTbl2) + require.Equal(t, statsTbl2, statsTbl1) } -func (s *testSerialStatsSuite) TestIssues24349(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestIssues24349(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") testKit.MustExec("set @@tidb_analyze_version=2") @@ -2984,9 +2995,10 @@ func (s *testSerialStatsSuite) TestIssues24349(c *C) { )) } -func (s *testStatsSuite) TestIssues24401(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestIssues24401(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") // normal table with static prune mode @@ -3012,18 +3024,19 @@ func (s *testStatsSuite) TestIssues24401(c *C) { testKit.MustExec("analyze table tp") rows := testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() lenRows := len(rows) - c.Assert(lenRows, Equals, 6) + require.Equal(t, 6, lenRows) // check fm-sketch won't increase infinitely testKit.MustExec("insert into t values (10), (20), (30), (12), (23), (23), (4344)") testKit.MustExec("analyze table tp") rows = testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() - c.Assert(len(rows), Equals, lenRows) + require.Len(t, rows, lenRows) } -func (s *testStatsSuite) TestIssues27147(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestIssues27147(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") @@ -3031,19 +3044,20 @@ func (s *testStatsSuite) TestIssues27147(c *C) { testKit.MustExec("create table t (a int, b int) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue);") testKit.MustExec("alter table t add index idx((a+5));") err := testKit.ExecToErr("analyze table t;") - c.Assert(err, Equals, nil) + require.Equal(t, nil, err) testKit.MustExec("drop table if exists t1") testKit.MustExec("create table t1 (a int, b int as (a+1) virtual, c int) partition by range (a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than maxvalue);") testKit.MustExec("alter table t1 add index idx((a+5));") err = testKit.ExecToErr("analyze table t1;") - c.Assert(err, Equals, nil) + require.Equal(t, nil, err) } -func (s *testStatsSuite) TestColumnCountFromStorage(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - do := s.do +func TestColumnCountFromStorage(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + do := dom h := do.StatsHandle() originLease := h.Lease() defer h.SetLease(originLease) @@ -3058,51 +3072,52 @@ func (s *testStatsSuite) TestColumnCountFromStorage(c *C) { is := do.InfoSchema() h = do.StatsHandle() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tt")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := tbl.Meta() h.TableStatsFromStorage(tblInfo, tblInfo.ID, false, 0) statsTbl := h.GetTableStats(tblInfo) - c.Assert(statsTbl.Columns[tblInfo.Columns[0].ID].Count, Equals, int64(2)) + require.Equal(t, int64(2), statsTbl.Columns[tblInfo.Columns[0].ID].Count) } -func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestIncrementalModifyCountUpdate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int)") tk.MustExec("set @@session.tidb_analyze_version = 2") - h := s.do.StatsHandle() + h := dom.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - tbl, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) tblInfo := tbl.Meta() tid := tblInfo.ID tk.MustExec("insert into t values(1),(2),(3)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - err = h.Update(s.do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + err = h.Update(dom.InfoSchema()) + require.NoError(t, err) tk.MustExec("analyze table t") tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows( "3 0", )) tk.MustExec("begin") - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) + txn, err := tk.Session().Txn(false) + require.NoError(t, err) startTS := txn.StartTS() tk.MustExec("commit") tk.MustExec("insert into t values(4),(5),(6)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - err = h.Update(s.do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + err = h.Update(dom.InfoSchema()) + require.NoError(t, err) // Simulate that the analyze would start before and finish after the second insert. - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS)), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectBaseCount", "return(3)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/injectBaseModifyCount", "return(0)"), IsNil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot", fmt.Sprintf("return(%d)", startTS))) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectBaseCount", "return(3)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/injectBaseModifyCount", "return(0)")) tk.MustExec("analyze table t") // Check the count / modify_count changes during the analyze are not lost. tk.MustQuery(fmt.Sprintf("select count, modify_count from mysql.stats_meta where table_id = %d", tid)).Check(testkit.Rows( @@ -3112,14 +3127,15 @@ func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d", tid)).Check(testkit.Rows( "3", )) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseCount"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/injectAnalyzeSnapshot")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseCount")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount")) } -func (s *testStatsSuite) TestRecordHistoricalStatsToStorage(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestRecordHistoricalStatsToStorage(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -3132,12 +3148,12 @@ func (s *testStatsSuite) TestRecordHistoricalStatsToStorage(c *C) { tk.MustExec("alter table t add index multi(a, b)") tk.MustExec("analyze table t with 2 topn") - tableInfo, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - require.NoError(c, err) - version, err := s.do.StatsHandle().RecordHistoricalStatsToStorage("t", tableInfo.Meta()) - require.NoError(c, err) + tableInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + version, err := dom.StatsHandle().RecordHistoricalStatsToStorage("t", tableInfo.Meta()) + require.NoError(t, err) rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where version = '%d'", version)).Rows() num, _ := strconv.Atoi(rows[0][0].(string)) - require.GreaterOrEqual(c, num, 1) + require.GreaterOrEqual(t, num, 1) } diff --git a/statistics/handle/main_test.go b/statistics/handle/main_test.go index 658b377aba2b4..9738db767eca3 100644 --- a/statistics/handle/main_test.go +++ b/statistics/handle/main_test.go @@ -23,7 +23,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/statistics/handle/update.go b/statistics/handle/update.go index d9921fe948b72..d1a1767ce2c5f 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -191,7 +191,7 @@ var ( MinLogErrorRate = atomic.NewFloat64(0.5) ) -// StoreQueryFeedback merges the feedback into stats collector. +// StoreQueryFeedback merges the feedback into stats collector. Deprecated. func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Handle) error { q := feedback.(*statistics.QueryFeedback) if !q.Valid || q.Hist == nil { @@ -468,6 +468,7 @@ func (h *Handle) DumpStatsDeltaToKV(mode dumpMode) error { deltaMap.update(id, -item.Delta, -item.Count, nil) } if err = h.dumpTableStatColSizeToKV(id, item); err != nil { + delete(deltaMap, id) return errors.Trace(err) } if updated { @@ -553,7 +554,7 @@ func (h *Handle) dumpTableStatColSizeToKV(id int64, delta variable.TableDelta) e return errors.Trace(err) } -// DumpStatsFeedbackToKV dumps the stats feedback to KV. +// DumpStatsFeedbackToKV dumps the stats feedback to KV. Deprecated. func (h *Handle) DumpStatsFeedbackToKV() error { h.feedback.Lock() feedback := h.feedback.data @@ -1114,7 +1115,9 @@ func (h *Handle) autoAnalyzeTable(tblInfo *model.TableInfo, statsTbl *statistics } func (h *Handle) autoAnalyzePartitionTable(tblInfo *model.TableInfo, pi *model.PartitionInfo, db string, start, end time.Time, ratio float64) bool { + h.mu.RLock() tableStatsVer := h.mu.ctx.GetSessionVars().AnalyzeVersion + h.mu.RUnlock() partitionNames := make([]interface{}, 0, len(pi.Definitions)) for _, def := range pi.Definitions { partitionStatsTbl := h.GetPartitionStats(tblInfo, def.ID) @@ -1334,7 +1337,7 @@ func logForPK(prefix string, c *statistics.Column, ranges []*ranger.Range, actua } } -// RecalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. +// RecalculateExpectCount recalculates the expect row count if the origin row count is estimated by pseudo. Deprecated. func (h *Handle) RecalculateExpectCount(q *statistics.QueryFeedback) error { t, ok := h.statsCache.Load().(statsCache).tables[q.PhysicalID] if !ok { @@ -1448,7 +1451,7 @@ func convertRangeType(ran *ranger.Range, ft *types.FieldType, loc *time.Location return statistics.ConvertDatumsType(ran.HighVal, ft, loc) } -// DumpFeedbackForIndex dumps the feedback for index. +// DumpFeedbackForIndex dumps the feedback for index. Deprecated. // For queries that contains both equality and range query, we will split them and Update accordingly. func (h *Handle) DumpFeedbackForIndex(q *statistics.QueryFeedback, t *statistics.Table) error { idx, ok := t.Indices[q.Hist.ID] diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index eea0a6f2db148..fddab89cdacc7 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -16,17 +16,12 @@ package handle_test import ( "fmt" - "math" "math/rand" - "os" "strconv" "strings" + "testing" "time" - . "github.com/pingcap/check" - "github.com/pingcap/log" - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -35,76 +30,20 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" dto "github.com/prometheus/client_model/go" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" ) -var _ = Suite(&testStatsSuite{}) -var _ = SerialSuites(&testSerialStatsSuite{}) - -type testSerialStatsSuite struct { - store kv.Storage - do *domain.Domain -} - -func (s *testSerialStatsSuite) SetUpSuite(c *C) { - testleak.BeforeTest() - // Add the hook here to avoid data race. - var err error - s.store, s.do, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testSerialStatsSuite) TearDownSuite(c *C) { - s.do.Close() - s.store.Close() - testleak.AfterTest(c)() -} - -type testSuiteBase struct { - store kv.Storage - do *domain.Domain - hook *logHook -} - -type testStatsSuite struct { - testSuiteBase -} - -func (s *testSuiteBase) SetUpSuite(c *C) { - testleak.BeforeTest() - // Add the hook here to avoid data race. - s.registerHook() - var err error - s.store, s.do, err = newStoreWithBootstrap() - c.Assert(err, IsNil) -} - -func (s *testSuiteBase) TearDownSuite(c *C) { - s.do.Close() - s.store.Close() - testleak.AfterTest(c)() -} - -func (s *testSuiteBase) registerHook() { - conf := &log.Config{Level: os.Getenv("log_level"), File: log.FileLogConfig{}} - _, r, _ := log.InitLogger(conf) - s.hook = &logHook{r.Core, ""} - lg := zap.New(s.hook) - log.ReplaceGlobals(lg, r) -} - -func (s *testStatsSuite) TestSingleSessionInsert(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestSingleSessionInsert(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 1") testKit.MustExec("create table t1 (c1 int, c2 int)") @@ -119,52 +58,52 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("insert into t2 values(1, 2)") } - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo1 := tbl1.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 := h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo2 := tbl2.Meta() stats2 := h.GetTableStats(tableInfo2) - c.Assert(stats2.Count, Equals, int64(rowCount2)) + require.Equal(t, int64(rowCount2), stats2.Count) testKit.MustExec("analyze table t1") // Test update in a txn. for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1*2)) + require.Equal(t, int64(rowCount1*2), stats1.Count) // Test IncreaseFactor. - count, err := stats1.ColumnEqualRowCount(testKit.Se, types.NewIntDatum(1), tableInfo1.Columns[0].ID) - c.Assert(err, IsNil) - c.Assert(count, Equals, float64(rowCount1*2)) + count, err := stats1.ColumnEqualRowCount(testKit.Session(), types.NewIntDatum(1), tableInfo1.Columns[0].ID) + require.NoError(t, err) + require.Equal(t, float64(rowCount1*2), count) testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") } testKit.MustExec("commit") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1*3)) + require.Equal(t, int64(rowCount1*3), stats1.Count) testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { @@ -177,20 +116,20 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("update t2 set c2 = c1") } testKit.MustExec("commit") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1*3)) + require.Equal(t, int64(rowCount1*3), stats1.Count) stats2 = h.GetTableStats(tableInfo2) - c.Assert(stats2.Count, Equals, int64(rowCount2)) + require.Equal(t, int64(rowCount2), stats2.Count) testKit.MustExec("begin") testKit.MustExec("delete from t1") testKit.MustExec("commit") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(0)) + require.Equal(t, int64(0), stats1.Count) rs := testKit.MustQuery("select modify_count from mysql.stats_meta") rs.Check(testkit.Rows("40", "70")) @@ -209,52 +148,54 @@ func (s *testStatsSuite) TestSingleSessionInsert(c *C) { testKit.MustExec("insert into t1 values (1,2)") } err = h.DumpStatsDeltaToKV(handle.DumpDelta) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) // not dumped testKit.MustExec("insert into t1 values (1,2)") err = h.DumpStatsDeltaToKV(handle.DumpDelta) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) h.FlushStats() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1+1)) + require.Equal(t, int64(rowCount1+1), stats1.Count) } -func (s *testStatsSuite) TestRollback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestRollback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("begin") testKit.MustExec("insert into t values (1,2)") testKit.MustExec("rollback") - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats := h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(0)) - c.Assert(stats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(0), stats.Count) + require.Equal(t, int64(0), stats.ModifyCount) } -func (s *testStatsSuite) TestMultiSession(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestMultiSession(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int, c2 int)") @@ -263,27 +204,27 @@ func (s *testStatsSuite) TestMultiSession(c *C) { testKit.MustExec("insert into t1 values(1, 2)") } - testKit1 := testkit.NewTestKit(c, s.store) + testKit1 := testkit.NewTestKit(t, store) for i := 0; i < rowCount1; i++ { testKit1.MustExec("insert into test.t1 values(1, 2)") } - testKit2 := testkit.NewTestKit(c, s.store) + testKit2 := testkit.NewTestKit(t, store) for i := 0; i < rowCount1; i++ { testKit2.MustExec("delete from test.t1 limit 1") } - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo1 := tbl1.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 := h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(1, 2)") @@ -297,124 +238,126 @@ func (s *testStatsSuite) TestMultiSession(c *C) { testKit2.MustExec("delete from test.t1 limit 1") } - testKit.Se.Close() - testKit2.Se.Close() + testKit.Session().Close() + testKit2.Session().Close() - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1*2)) - // The session in testKit is already Closed, set it to nil will create a new session. - testKit.Se = nil + require.Equal(t, int64(rowCount1*2), stats1.Count) + testKit.RefreshSession() rs := testKit.MustQuery("select modify_count from mysql.stats_meta") rs.Check(testkit.Rows("60")) } -func (s *testStatsSuite) TestTxnWithFailure(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestTxnWithFailure(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t1 (c1 int primary key, c2 int)") - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo1 := tbl1.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) rowCount1 := 10 testKit.MustExec("begin") for i := 0; i < rowCount1; i++ { testKit.MustExec("insert into t1 values(?, 2)", i) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 := h.GetTableStats(tableInfo1) // have not commit - c.Assert(stats1.Count, Equals, int64(0)) + require.Equal(t, int64(0), stats1.Count) testKit.MustExec("commit") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) _, err = testKit.Exec("insert into t1 values(0, 2)") - c.Assert(err, NotNil) + require.Error(t, err) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1)) + require.Equal(t, int64(rowCount1), stats1.Count) testKit.MustExec("insert into t1 values(-1, 2)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) stats1 = h.GetTableStats(tableInfo1) - c.Assert(stats1.Count, Equals, int64(rowCount1+1)) + require.Equal(t, int64(rowCount1+1), stats1.Count) } -func (s *testStatsSuite) TestUpdatePartition(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(string(s.do.StatsHandle().CurrentPruneMode()))) +func TestUpdatePartition(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + testKit.MustQuery("select @@tidb_partition_prune_mode").Check(testkit.Rows(string(dom.StatsHandle().CurrentPruneMode()))) testKit.MustExec("use test") testkit.WithPruneMode(testKit, variable.Static, func() { - err := s.do.StatsHandle().RefreshVars() - c.Assert(err, IsNil) + err := dom.StatsHandle().RefreshVars() + require.NoError(t, err) testKit.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b char(5)) PARTITION BY RANGE (a) (PARTITION p0 VALUES LESS THAN (6),PARTITION p1 VALUES LESS THAN (11))` testKit.MustExec(createTable) - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) pi := tableInfo.GetPartitionInfo() - c.Assert(len(pi.Definitions), Equals, 2) + require.Len(t, pi.Definitions, 2) bColID := tableInfo.Columns[1].ID testKit.MustExec(`insert into t values (1, "a"), (7, "a")`) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(1)) - c.Assert(statsTbl.Count, Equals, int64(1)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(2)) + require.Equal(t, int64(1), statsTbl.ModifyCount) + require.Equal(t, int64(1), statsTbl.Count) + require.Equal(t, int64(2), statsTbl.Columns[bColID].TotColSize) } testKit.MustExec(`update t set a = a + 1, b = "aa"`) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(2)) - c.Assert(statsTbl.Count, Equals, int64(1)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(3)) + require.Equal(t, int64(2), statsTbl.ModifyCount) + require.Equal(t, int64(1), statsTbl.Count) + require.Equal(t, int64(3), statsTbl.Columns[bColID].TotColSize) } testKit.MustExec("delete from t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) for _, def := range pi.Definitions { statsTbl := h.GetPartitionStats(tableInfo, def.ID) - c.Assert(statsTbl.ModifyCount, Equals, int64(3)) - c.Assert(statsTbl.Count, Equals, int64(0)) - c.Assert(statsTbl.Columns[bColID].TotColSize, Equals, int64(0)) + require.Equal(t, int64(3), statsTbl.ModifyCount) + require.Equal(t, int64(0), statsTbl.Count) + require.Equal(t, int64(0), statsTbl.Columns[bColID].TotColSize) } }) } -func (s *testStatsSuite) TestAutoUpdate(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestAutoUpdate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("create table t (a varchar(20))") @@ -426,31 +369,31 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) stats := h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(0)) + require.Equal(t, int64(0), stats.Count) _, err = testKit.Exec("insert into t values ('ss'), ('ss'), ('ss'), ('ss'), ('ss')") - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(5)) - c.Assert(stats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(5), stats.Count) + require.Equal(t, int64(0), stats.ModifyCount) for _, item := range stats.Columns { // TotColSize = 5*(2(length of 'ss') + 1(size of len byte)). - c.Assert(item.TotColSize, Equals, int64(15)) + require.Equal(t, int64(15), item.TotColSize) break } @@ -458,64 +401,65 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { h.SetLease(time.Second) defer func() { h.SetLease(0) }() _, err = testKit.Exec("insert into t values ('fff')") - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(6)) - c.Assert(stats.ModifyCount, Equals, int64(1)) + require.Equal(t, int64(6), stats.Count) + require.Equal(t, int64(1), stats.ModifyCount) _, err = testKit.Exec("insert into t values ('fff')") - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(7)) - c.Assert(stats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(7), stats.Count) + require.Equal(t, int64(0), stats.ModifyCount) _, err = testKit.Exec("insert into t values ('eee')") - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(8)) + require.Equal(t, int64(8), stats.Count) // Modify count is non-zero means that we do not analyze the table. - c.Assert(stats.ModifyCount, Equals, int64(1)) + require.Equal(t, int64(1), stats.ModifyCount) for _, item := range stats.Columns { // TotColSize = 27, because the table has not been analyzed, and insert statement will add 3(length of 'eee') to TotColSize. - c.Assert(item.TotColSize, Equals, int64(27)) + require.Equal(t, int64(27), item.TotColSize) break } testKit.MustExec("analyze table t") _, err = testKit.Exec("create index idx on t(a)") - c.Assert(err, IsNil) + require.NoError(t, err) is = do.InfoSchema() tbl, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo = tbl.Meta() h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats = h.GetTableStats(tableInfo) - c.Assert(stats.Count, Equals, int64(8)) - c.Assert(stats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(8), stats.Count) + require.Equal(t, int64(0), stats.ModifyCount) hg, ok := stats.Indices[tableInfo.Indices[0].ID] - c.Assert(ok, IsTrue) - c.Assert(hg.NDV, Equals, int64(3)) - c.Assert(hg.Len(), Equals, 0) - c.Assert(hg.TopN.Num(), Equals, 3) + require.True(t, ok) + require.Equal(t, int64(3), hg.NDV) + require.Equal(t, 0, hg.Len()) + require.Equal(t, 3, hg.TopN.Num()) }) } -func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestAutoUpdatePartition(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testkit.WithPruneMode(testKit, variable.Static, func() { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -529,32 +473,33 @@ func (s *testStatsSuite) TestAutoUpdatePartition(c *C) { testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() pi := tableInfo.GetPartitionInfo() h := do.StatsHandle() - c.Assert(h.RefreshVars(), IsNil) + require.NoError(t, h.RefreshVars()) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) stats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) - c.Assert(stats.Count, Equals, int64(0)) + require.Equal(t, int64(0), stats.Count) testKit.MustExec("insert into t values (1)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) stats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) - c.Assert(stats.Count, Equals, int64(1)) - c.Assert(stats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(1), stats.Count) + require.Equal(t, int64(0), stats.ModifyCount) }) } -func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAutoAnalyzeOnEmptyTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) @@ -563,12 +508,12 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() - t := time.Now().Add(-1 * time.Minute) - h, m := t.Hour(), t.Minute() + tt := time.Now().Add(-1 * time.Minute) + h, m := tt.Hour(), tt.Minute() start, end := fmt.Sprintf("%02d:%02d +0000", h, m), fmt.Sprintf("%02d:%02d +0000", h, m) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", start)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", end)) - s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()) + dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()) tk.MustExec("use test") tk.MustExec("create table t (a int, index idx(a))") @@ -576,20 +521,21 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { tk.MustExec("analyze table t") // to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(handle.AutoAnalyzeMinCnt))) - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) // test if it will be limited by the time range - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) + require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) + require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) } -func (s *testSerialStatsSuite) TestAutoAnalyzeOutOfSpecifiedTime(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAutoAnalyzeOutOfSpecifiedTime(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) @@ -598,12 +544,12 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOutOfSpecifiedTime(c *C) { tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() - t := time.Now().Add(-1 * time.Minute) - h, m := t.Hour(), t.Minute() + tt := time.Now().Add(-1 * time.Minute) + h, m := tt.Hour(), tt.Minute() start, end := fmt.Sprintf("%02d:%02d +0000", h, m), fmt.Sprintf("%02d:%02d +0000", h, m) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='%v'", start)) tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", end)) - s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()) + dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema()) tk.MustExec("use test") tk.MustExec("create table t (a int)") @@ -611,23 +557,24 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOutOfSpecifiedTime(c *C) { tk.MustExec("analyze table t") // to pass the AutoAnalyzeMinCnt check in autoAnalyzeTable tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", int(handle.AutoAnalyzeMinCnt))) - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) + require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) tk.MustExec("analyze table t") tk.MustExec("alter table t add index ia(a)") - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) + require.False(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) + require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) } -func (s *testSerialStatsSuite) TestIssue25700(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestIssue25700(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) defer func() { @@ -642,108 +589,110 @@ func (s *testSerialStatsSuite) TestIssue25700(c *C) { tk.MustExec("CREATE TABLE `t` ( `ldecimal` decimal(32,4) DEFAULT NULL, `rdecimal` decimal(32,4) DEFAULT NULL, `gen_col` decimal(36,4) GENERATED ALWAYS AS (`ldecimal` + `rdecimal`) VIRTUAL, `col_timestamp` timestamp(3) NULL DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;") tk.MustExec("analyze table t") tk.MustExec("INSERT INTO `t` (`ldecimal`, `rdecimal`, `col_timestamp`) VALUES (2265.2200, 9843.4100, '1999-12-31 16:00:00')" + strings.Repeat(", (2265.2200, 9843.4100, '1999-12-31 16:00:00')", int(handle.AutoAnalyzeMinCnt))) - c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(s.do.StatsHandle().Update(s.do.InfoSchema()), IsNil) + require.NoError(t, dom.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, dom.StatsHandle().Update(dom.InfoSchema())) - c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) - c.Assert(tk.MustQuery("show analyze status").Rows()[1][7], Equals, "finished") + require.True(t, dom.StatsHandle().HandleAutoAnalyze(dom.InfoSchema())) + require.Equal(t, "finished", tk.MustQuery("show analyze status").Rows()[1][7]) } -func (s *testSerialStatsSuite) TestAutoAnalyzeOnChangeAnalyzeVer(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAutoAnalyzeOnChangeAnalyzeVer(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("create table t(a int, index idx(a))") tk.MustExec("insert into t values(1)") tk.MustExec("set @@global.tidb_analyze_version = 1") - do := s.do + do := dom handle.AutoAnalyzeMinCnt = 0 defer func() { handle.AutoAnalyzeMinCnt = 1000 }() h := do.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) is := do.InfoSchema() err = h.UpdateSessionVar() - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) // Auto analyze when global ver is 1. h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) statsTbl1 := h.GetTableStats(tbl.Meta()) // Check that all the version of t's stats are 1. for _, col := range statsTbl1.Columns { - c.Assert(col.StatsVer, Equals, int64(1)) + require.Equal(t, int64(1), col.StatsVer) } for _, idx := range statsTbl1.Indices { - c.Assert(idx.StatsVer, Equals, int64(1)) + require.Equal(t, int64(1), idx.StatsVer) } tk.MustExec("set @@global.tidb_analyze_version = 2") err = h.UpdateSessionVar() - c.Assert(err, IsNil) + require.NoError(t, err) tk.MustExec("insert into t values(1), (2), (3), (4)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) // Auto analyze t whose version is 1 after setting global ver to 2. h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl1 = h.GetTableStats(tbl.Meta()) - c.Assert(statsTbl1.Count, Equals, int64(5)) + require.Equal(t, int64(5), statsTbl1.Count) // All of its statistics should still be version 1. for _, col := range statsTbl1.Columns { - c.Assert(col.StatsVer, Equals, int64(1)) + require.Equal(t, int64(1), col.StatsVer) } for _, idx := range statsTbl1.Indices { - c.Assert(idx.StatsVer, Equals, int64(1)) + require.Equal(t, int64(1), idx.StatsVer) } // Add a new table after the analyze version set to 2. tk.MustExec("create table tt(a int, index idx(a))") tk.MustExec("insert into tt values(1), (2), (3), (4), (5)") err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) is = do.InfoSchema() tbl2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("tt")) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, err) + require.NoError(t, h.Update(is)) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl2 := h.GetTableStats(tbl2.Meta()) // Since it's a newly created table. Auto analyze should analyze it's statistics to version2. for _, idx := range statsTbl2.Indices { - c.Assert(idx.StatsVer, Equals, int64(2)) + require.Equal(t, int64(2), idx.StatsVer) } for _, col := range statsTbl2.Columns { - c.Assert(col.StatsVer, Equals, int64(2)) + require.Equal(t, int64(2), col.StatsVer) } tk.MustExec("set @@global.tidb_analyze_version = 1") } -func (s *testStatsSuite) TestTableAnalyzed(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestTableAnalyzed(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1)") - is := s.do.InfoSchema() + is := dom.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl := h.GetTableStats(tableInfo) - c.Assert(handle.TableAnalyzed(statsTbl), IsFalse) + require.False(t, handle.TableAnalyzed(statsTbl)) testKit.MustExec("analyze table t") - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl = h.GetTableStats(tableInfo) - c.Assert(handle.TableAnalyzed(statsTbl), IsTrue) + require.True(t, handle.TableAnalyzed(statsTbl)) h.Clear() oriLease := h.Lease() @@ -752,17 +701,18 @@ func (s *testStatsSuite) TestTableAnalyzed(c *C) { defer func() { h.SetLease(oriLease) }() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) statsTbl = h.GetTableStats(tableInfo) - c.Assert(handle.TableAnalyzed(statsTbl), IsTrue) + require.True(t, handle.TableAnalyzed(statsTbl)) } -func (s *testStatsSuite) TestUpdateErrorRate(c *C) { - defer cleanEnv(c, s.store, s.do) - h := s.do.StatsHandle() - is := s.do.InfoSchema() +func TestUpdateErrorRate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + h := dom.StatsHandle() + is := dom.InfoSchema() h.SetLease(0) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() oriErrorRate := handle.MinLogErrorRate.Load() @@ -775,71 +725,72 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { handle.MinLogScanCount.Store(0) handle.MinLogErrorRate.Store(0) - testKit := testkit.NewTestKit(c, s.store) + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 0") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) testKit.MustExec("insert into t values (1, 3)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2, 3)") testKit.MustExec("insert into t values (5, 3)") testKit.MustExec("insert into t values (8, 3)") testKit.MustExec("insert into t values (12, 3)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - is = s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + is = dom.InfoSchema() + require.NoError(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) aID := tblInfo.Columns[0].ID bID := tblInfo.Indices[0].ID // The statistic table is outdated now. - c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) + require.True(t, tbl.Columns[aID].NotAccurate()) testKit.MustQuery("select * from t where a between 1 and 10") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(is)) h.UpdateErrorRate(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl = h.GetTableStats(tblInfo) // The error rate of this column is not larger than MaxErrorRate now. - c.Assert(tbl.Columns[aID].NotAccurate(), IsFalse) + require.False(t, tbl.Columns[aID].NotAccurate()) - c.Assert(tbl.Indices[bID].NotAccurate(), IsTrue) + require.True(t, tbl.Indices[bID].NotAccurate()) testKit.MustQuery("select * from t where b between 2 and 10") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(is)) h.UpdateErrorRate(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl = h.GetTableStats(tblInfo) - c.Assert(tbl.Indices[bID].NotAccurate(), IsFalse) - c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(1)) + require.False(t, tbl.Indices[bID].NotAccurate()) + require.Equal(t, int64(1), tbl.Indices[bID].QueryTotal) testKit.MustExec("analyze table t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) tbl = h.GetTableStats(tblInfo) - c.Assert(tbl.Indices[bID].QueryTotal, Equals, int64(0)) + require.Equal(t, int64(0), tbl.Indices[bID].QueryTotal) } -func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { - defer cleanEnv(c, s.store, s.do) - h := s.do.StatsHandle() - is := s.do.InfoSchema() +func TestUpdatePartitionErrorRate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + h := dom.StatsHandle() + is := dom.InfoSchema() h.SetLease(0) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() oriErrorRate := handle.MinLogErrorRate.Load() @@ -852,46 +803,46 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { handle.MinLogScanCount.Store(0) handle.MinLogErrorRate.Store(0) - testKit := testkit.NewTestKit(c, s.store) + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) testKit.MustExec("create table t (a bigint(64), primary key(a)) partition by range (a) (partition p0 values less than (30))") err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) testKit.MustExec("insert into t values (1)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (2)") testKit.MustExec("insert into t values (5)") testKit.MustExec("insert into t values (8)") testKit.MustExec("insert into t values (12)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - is = s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + is = dom.InfoSchema() + require.NoError(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() pid := tblInfo.Partition.Definitions[0].ID tbl := h.GetPartitionStats(tblInfo, pid) aID := tblInfo.Columns[0].ID // The statistic table is outdated now. - c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) + require.True(t, tbl.Columns[aID].NotAccurate()) testKit.MustQuery("select * from t where a between 1 and 10") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(is)) h.UpdateErrorRate(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl = h.GetPartitionStats(tblInfo, pid) // Feedback will not take effect under partition table. - c.Assert(tbl.Columns[aID].NotAccurate(), IsTrue) + require.True(t, tbl.Columns[aID].NotAccurate()) } func appendBucket(h *statistics.Histogram, l, r int64) { @@ -899,7 +850,7 @@ func appendBucket(h *statistics.Histogram, l, r int64) { h.AppendBucket(&lower, &upper, 0, 0) } -func (s *testStatsSuite) TestSplitRange(c *C) { +func TestSplitRange(t *testing.T) { h := statistics.NewHistogram(0, 0, 0, 0, types.NewFieldType(mysql.TypeLong), 5, 0) appendBucket(h, 1, 1) appendBucket(h, 2, 5) @@ -934,14 +885,14 @@ func (s *testStatsSuite) TestSplitRange(c *C) { result: "[8,9)", }, } - for _, t := range tests { - ranges := make([]*ranger.Range, 0, len(t.points)/2) - for i := 0; i < len(t.points); i += 2 { + for _, test := range tests { + ranges := make([]*ranger.Range, 0, len(test.points)/2) + for i := 0; i < len(test.points); i += 2 { ranges = append(ranges, &ranger.Range{ - LowVal: []types.Datum{types.NewIntDatum(t.points[i])}, - LowExclude: t.exclude[i], - HighVal: []types.Datum{types.NewIntDatum(t.points[i+1])}, - HighExclude: t.exclude[i+1], + LowVal: []types.Datum{types.NewIntDatum(test.points[i])}, + LowExclude: test.exclude[i], + HighVal: []types.Datum{types.NewIntDatum(test.points[i+1])}, + HighExclude: test.exclude[i+1], Collators: collate.GetBinaryCollatorSlice(1), }) } @@ -950,13 +901,14 @@ func (s *testStatsSuite) TestSplitRange(c *C) { for _, ran := range ranges { ranStrs = append(ranStrs, ran.String()) } - c.Assert(strings.Join(ranStrs, ","), Equals, t.result) + require.Equal(t, test.result, strings.Join(ranStrs, ",")) } } -func (s *testStatsSuite) TestQueryFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestQueryFeedback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 0") testKit.MustExec("create table t (a bigint(64), b bigint(64), primary key(a), index idx(b))") @@ -964,7 +916,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { testKit.MustExec("analyze table t with 0 topn") testKit.MustExec("insert into t values (3,4)") - h := s.do.StatsHandle() + h := dom.StatsHandle() oriProbability := statistics.FeedbackProbability.Load() oriNumber := statistics.MaxNumberOfRanges oriMinLogCount := handle.MinLogScanCount.Load() @@ -1009,73 +961,74 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { idxCols: 1, }, } - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + for i, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, err) + require.NoError(t, h.Update(is)) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) - if t.idxCols == 0 { - c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[i].hist) + if test.idxCols == 0 { + require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) } else { - c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[i].hist) + require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) } } // Feedback from limit executor may not be accurate. testKit.MustQuery("select * from t where t.a <= 5 limit 1") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) feedback := h.GetQueryFeedback() - c.Assert(feedback.Size, Equals, 0) + require.Equal(t, 0, feedback.Size) // Test only collect for max number of Ranges. statistics.MaxNumberOfRanges = 0 - for _, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + for _, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) feedback := h.GetQueryFeedback() - c.Assert(feedback.Size, Equals, 0) + require.Equal(t, 0, feedback.Size) } // Test collect feedback by probability. statistics.FeedbackProbability.Store(0) statistics.MaxNumberOfRanges = oriNumber - for _, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + for _, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) feedback := h.GetQueryFeedback() - c.Assert(feedback.Size, Equals, 0) + require.Equal(t, 0, feedback.Size) } // Test that after drop stats, the feedback won't cause panic. statistics.FeedbackProbability.Store(1) - for _, t := range tests { - testKit.MustQuery(t.sql) + for _, test := range tests { + testKit.MustQuery(test.sql) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) testKit.MustExec("drop stats t") - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) // Test that the outdated feedback won't cause panic. testKit.MustExec("analyze table t") - for _, t := range tests { - testKit.MustQuery(t.sql) + for _, test := range tests { + testKit.MustQuery(test.sql) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) testKit.MustExec("drop table t") - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) } -func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestQueryFeedbackForPartition(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 0") testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) @@ -1098,7 +1051,7 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { handle.MinLogScanCount.Store(0) handle.MinLogErrorRate.Store(0) - h := s.do.StatsHandle() + h := dom.StatsHandle() // Feedback will not take effect under partition table. tests := []struct { sql string @@ -1128,12 +1081,12 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { idxCols: 1, }, } - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() pi := tblInfo.GetPartitionInfo() - c.Assert(pi, NotNil) + require.NotNil(t, pi) // This test will check the result of partition p0. var pid int64 @@ -1144,73 +1097,76 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { } } - for i, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + for i, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, err) + require.NoError(t, h.Update(is)) tbl := h.GetPartitionStats(tblInfo, pid) - if t.idxCols == 0 { - c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, tests[i].hist) + if test.idxCols == 0 { + require.Equal(t, tests[i].hist, tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) } else { - c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, tests[i].hist) + require.Equal(t, tests[i].hist, tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) } } testKit.MustExec("drop table t") } -func (s *testStatsSuite) TestUpdateSystemTable(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestUpdateSystemTable(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("insert into t values (1,2)") testKit.MustExec("analyze table t") testKit.MustExec("analyze table mysql.stats_histograms") - h := s.do.StatsHandle() - c.Assert(h.Update(s.do.InfoSchema()), IsNil) + h := dom.StatsHandle() + require.NoError(t, h.Update(dom.InfoSchema())) feedback := h.GetQueryFeedback() // We may have query feedback for system tables, but we do not need to store them. - c.Assert(feedback.Size, Equals, 0) + require.Equal(t, 0, feedback.Size) } -func (s *testStatsSuite) TestOutOfOrderUpdate(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestOutOfOrderUpdate(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int)") testKit.MustExec("insert into t values (1,2)") - do := s.do + do := dom is := do.InfoSchema() tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() h := do.StatsHandle() err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) // Simulate the case that another tidb has inserted some value, but delta info has not been dumped to kv yet. testKit.MustExec("insert into t values (2,2),(4,5)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 1 where table_id = %d", tableInfo.ID)) testKit.MustExec("delete from t") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("1")) // Now another tidb has updated the delta info. testKit.MustExec(fmt.Sprintf("update mysql.stats_meta set count = 3 where table_id = %d", tableInfo.ID)) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustQuery("select count from mysql.stats_meta").Check(testkit.Rows("0")) } -func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestUpdateStatsByLocalFeedback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 0") testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) @@ -1218,7 +1174,7 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t with 0 topn") testKit.MustExec("insert into t values (3,5)") - h := s.do.StatsHandle() + h := dom.StatsHandle() oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() oriErrorRate := handle.MinLogErrorRate.Load() @@ -1233,9 +1189,9 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { handle.MinLogScanCount.Store(0) handle.MinLogErrorRate.Store(0) - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() h.GetTableStats(tblInfo) @@ -1244,34 +1200,35 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { testKit.MustQuery("select * from t where a > 1") testKit.MustQuery("select * from t use index(idx) where b = 5") - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) tbl := h.GetTableStats(tblInfo) - c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, "column:1 ndv:3 totColSize:0\n"+ + require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ "num: 2 lower_bound: 2 upper_bound: 4 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 9223372036854775807 repeats: 0 ndv: 0") + "num: 1 lower_bound: 4 upper_bound: 9223372036854775807 repeats: 0 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) sc := &stmtctx.StatementContext{TimeZone: time.Local} low, err := codec.EncodeKey(sc, nil, types.NewIntDatum(5)) - c.Assert(err, IsNil) + require.NoError(t, err) - c.Assert(tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low), Equals, uint64(2)) + require.Equal(t, uint64(2), tbl.Indices[tblInfo.Indices[0].ID].CMSketch.QueryBytes(low)) - c.Assert(tbl.Indices[tblInfo.Indices[0].ID].ToString(1), Equals, "index:1 ndv:2\n"+ + require.Equal(t, "index:1 ndv:2\n"+ "num: 2 lower_bound: -inf upper_bound: 5 repeats: 0 ndv: 0\n"+ - "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0") + "num: 1 lower_bound: 5 upper_bound: 5 repeats: 1 ndv: 0", tbl.Indices[tblInfo.Indices[0].ID].ToString(1)) // Test that it won't cause panic after update. testKit.MustQuery("select * from t use index(idx) where b > 0") // Test that after drop stats, it won't cause panic. testKit.MustExec("drop stats t") - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) } -func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestUpdatePartitionStatsByLocalFeedback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set @@session.tidb_analyze_version = 0") testKit.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) @@ -1279,7 +1236,7 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { testKit.MustExec("insert into t values (1,2),(2,2),(4,5)") testKit.MustExec("analyze table t") testKit.MustExec("insert into t values (3,5)") - h := s.do.StatsHandle() + h := dom.StatsHandle() oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() oriErrorRate := handle.MinLogErrorRate.Load() @@ -1292,28 +1249,29 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { handle.MinLogScanCount.Store(0) handle.MinLogErrorRate.Store(0) - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) testKit.MustQuery("select * from t where a > 1").Check(testkit.Rows("2 2", "3 5", "4 5")) - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) tblInfo := table.Meta() pid := tblInfo.Partition.Definitions[0].ID tbl := h.GetPartitionStats(tblInfo, pid) // Feedback will not take effect under partition table. - c.Assert(tbl.Columns[tblInfo.Columns[0].ID].ToString(0), Equals, "column:1 ndv:3 totColSize:0\n"+ + require.Equal(t, "column:1 ndv:3 totColSize:0\n"+ "num: 1 lower_bound: 1 upper_bound: 1 repeats: 1 ndv: 0\n"+ "num: 1 lower_bound: 2 upper_bound: 2 repeats: 1 ndv: 0\n"+ - "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0") + "num: 1 lower_bound: 4 upper_bound: 4 repeats: 1 ndv: 0", tbl.Columns[tblInfo.Columns[0].ID].ToString(0)) } -func (s *testStatsSuite) TestFeedbackWithStatsVer2(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestFeedbackWithStatsVer2(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("set global tidb_analyze_version = 1") testKit.MustExec("set @@tidb_analyze_version = 1") @@ -1344,14 +1302,14 @@ func (s *testStatsSuite) TestFeedbackWithStatsVer2(c *C) { testKit.MustExec("insert into t values (1,2),(2,2),(4,5),(2,3),(3,4)") } testKit.MustExec("analyze table t with 0 topn") - h := s.do.StatsHandle() - is := s.do.InfoSchema() + h := dom.StatsHandle() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() testKit.MustExec("analyze table t") - err = h.Update(s.do.InfoSchema()) - c.Assert(err, IsNil) + err = h.Update(dom.InfoSchema()) + require.NoError(t, err) statsTblBefore := h.GetTableStats(tblInfo) statistics.FeedbackProbability.Store(1) // make the statistics inaccurate. @@ -1362,14 +1320,14 @@ func (s *testStatsSuite) TestFeedbackWithStatsVer2(c *C) { testKit.MustExec("select * from t where t.a <= 5 order by a desc") testKit.MustExec("select b from t use index(idx) where t.b <= 5") - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) err = h.DumpStatsFeedbackToKV() - c.Assert(err, IsNil) - err = h.HandleUpdateStats(s.do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, err) + err = h.HandleUpdateStats(dom.InfoSchema()) + require.NoError(t, err) statsTblAfter := h.GetTableStats(tblInfo) // assert that statistics not changed - assertTableEqual(c, statsTblBefore, statsTblAfter) + assertTableEqual(t, statsTblBefore, statsTblAfter) // Case 3: Feedback is still effective on version 1 statistics. testKit.MustExec("set tidb_analyze_version = 1") @@ -1382,22 +1340,22 @@ func (s *testStatsSuite) TestFeedbackWithStatsVer2(c *C) { for i := 0; i < 200; i++ { testKit.MustExec("insert into t1 values (3,4), (3,4), (3,4), (3,4), (3,4)") } - is = s.do.InfoSchema() + is = dom.InfoSchema() table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo = table.Meta() statsTblBefore = h.GetTableStats(tblInfo) // trigger feedback testKit.MustExec("select b from t1 use index(idx) where t1.b <= 5") - h.UpdateStatsByLocalFeedback(s.do.InfoSchema()) + h.UpdateStatsByLocalFeedback(dom.InfoSchema()) err = h.DumpStatsFeedbackToKV() - c.Assert(err, IsNil) - err = h.HandleUpdateStats(s.do.InfoSchema()) - c.Assert(err, IsNil) + require.NoError(t, err) + err = h.HandleUpdateStats(dom.InfoSchema()) + require.NoError(t, err) statsTblAfter = h.GetTableStats(tblInfo) // assert that statistics changed(feedback worked) - c.Assert(statistics.HistogramEqual(&statsTblBefore.Indices[1].Histogram, &statsTblAfter.Indices[1].Histogram, false), IsFalse) + require.False(t, statistics.HistogramEqual(&statsTblBefore.Indices[1].Histogram, &statsTblAfter.Indices[1].Histogram, false)) // Case 4: When existing version 1 stats + tidb_analyze_version=2 + feedback enabled, explicitly running `analyze table` still results in version 1 stats. statistics.FeedbackProbability.Store(0) @@ -1412,110 +1370,7 @@ func (s *testStatsSuite) TestFeedbackWithStatsVer2(c *C) { testKit.MustExec("set global tidb_analyze_version = 1") } -type logHook struct { - zapcore.Core - results string -} - -func (h *logHook) Write(entry zapcore.Entry, fields []zapcore.Field) error { - message := entry.Message - if idx := strings.Index(message, "[stats"); idx != -1 { - h.results = h.results + message - for _, f := range fields { - h.results = h.results + ", " + f.Key + "=" + h.field2String(f) - } - } - return nil -} - -func (h *logHook) field2String(field zapcore.Field) string { - switch field.Type { - case zapcore.StringType: - return field.String - case zapcore.Int64Type, zapcore.Int32Type, zapcore.Uint32Type, zapcore.Uint64Type: - return fmt.Sprintf("%v", field.Integer) - case zapcore.Float64Type: - return fmt.Sprintf("%v", math.Float64frombits(uint64(field.Integer))) - case zapcore.StringerType: - return field.Interface.(fmt.Stringer).String() - } - return "not support" -} - -func (h *logHook) Check(e zapcore.Entry, ce *zapcore.CheckedEntry) *zapcore.CheckedEntry { - if h.Enabled(e.Level) { - return ce.AddCore(e, h) - } - return ce -} - -func (s *testStatsSuite) TestLogDetailedInfo(c *C) { - c.Skip("not stable") - defer cleanEnv(c, s.store, s.do) - - oriProbability := statistics.FeedbackProbability.Load() - oriMinLogCount := handle.MinLogScanCount.Load() - oriMinError := handle.MinLogErrorRate.Load() - oriLevel := log.GetLevel() - oriLease := s.do.StatsHandle().Lease() - defer func() { - statistics.FeedbackProbability.Store(oriProbability) - handle.MinLogScanCount.Store(oriMinLogCount) - handle.MinLogErrorRate.Store(oriMinError) - s.do.StatsHandle().SetLease(oriLease) - log.SetLevel(oriLevel) - }() - statistics.FeedbackProbability.Store(1) - handle.MinLogScanCount.Store(0) - handle.MinLogErrorRate.Store(0) - s.do.StatsHandle().SetLease(1) - - testKit := testkit.NewTestKit(c, s.store) - testKit.MustExec("use test") - testKit.MustExec("drop table if exists t") - testKit.MustExec("set @@session.tidb_analyze_version=1") - testKit.MustExec("set @@session.tidb_stats_load_sync_wait =0") - testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), primary key(a), index idx(b), index idx_ba(b,a), index idx_bc(b,c))") - for i := 0; i < 20; i++ { - testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d, %d)", i, i, i)) - } - testKit.MustExec("analyze table t with 4 buckets") - tests := []struct { - sql string - result string - }{ - { - sql: "select * from t where t.a <= 15", - result: "[stats-feedback] test.t, column=a, rangeStr=range: [-inf,8), actual: 8, expected: 8, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1 ndv: 0, num: 8 lower_bound: 8 upper_bound: 15 repeats: 1 ndv: 0}" + - "[stats-feedback] test.t, column=a, rangeStr=range: [8,15), actual: 8, expected: 7, buckets: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1 ndv: 0}", - }, - { - sql: "select * from t use index(idx) where t.b <= 15", - result: "[stats-feedback] test.t, index=idx, rangeStr=range: [-inf,8), actual: 8, expected: 8, histogram: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1 ndv: 0, num: 8 lower_bound: 8 upper_bound: 15 repeats: 1 ndv: 0}" + - "[stats-feedback] test.t, index=idx, rangeStr=range: [8,16), actual: 8, expected: 8, histogram: {num: 8 lower_bound: 8 upper_bound: 15 repeats: 1 ndv: 0, num: 4 lower_bound: 16 upper_bound: 19 repeats: 1 ndv: 0}", - }, - { - sql: "select b from t use index(idx_ba) where b = 1 and a <= 5", - result: "[stats-feedback] test.t, index=idx_ba, actual=1, equality=1, expected equality=1, range=range: [-inf,6], actual: -1, expected: 6, buckets: {num: 8 lower_bound: 0 upper_bound: 7 repeats: 1 ndv: 0}", - }, - { - sql: "select b from t use index(idx_bc) where b = 1 and c <= 5", - result: "[stats-feedback] test.t, index=idx_bc, actual=1, equality=1, expected equality=1, range=[-inf,6], pseudo count=7", - }, - { - sql: "select b from t use index(idx_ba) where b = 1", - result: "[stats-feedback] test.t, index=idx_ba, rangeStr=value: 1, actual: 1, expected: 1", - }, - } - log.SetLevel(zapcore.DebugLevel) - for _, t := range tests { - s.hook.results = "" - testKit.MustQuery(t.sql) - c.Assert(s.hook.results, Equals, t.result) - } -} - -func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { +func TestNeedAnalyzeTable(t *testing.T) { columns := map[int64]*statistics.Column{} columns[1] = &statistics.Column{Count: 1} tests := []struct { @@ -1592,15 +1447,16 @@ func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { } for _, test := range tests { needAnalyze, reason := handle.NeedAnalyzeTable(test.tbl, test.limit, test.ratio) - c.Assert(needAnalyze, Equals, test.result) - c.Assert(strings.HasPrefix(reason, test.reason), IsTrue) + require.Equal(t, test.result, needAnalyze) + require.True(t, strings.HasPrefix(reason, test.reason)) } } -func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { - c.Skip("support update the topn of index equal conditions") - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestIndexQueryFeedback(t *testing.T) { + t.Skip("support update the topn of index equal conditions") + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() defer func() { @@ -1615,19 +1471,19 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) } - h := s.do.StatsHandle() + h := dom.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t with 3 buckets") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf(`insert into t values (1, %d, %d, %d, %d, %d, %d, "%s")`, i, i, i, i, i, i, fmt.Sprintf("1000-01-%02d", i+1))) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - is := s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + is := dom.InfoSchema() + require.NoError(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() tests := []struct { sql string @@ -1715,27 +1571,28 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { eqCount: 32, }, } - for i, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(h.Update(is), IsNil) + for i, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, h.Update(is)) tbl := h.GetTableStats(tblInfo) - if t.idxCols == 0 { - c.Assert(tbl.Columns[t.rangeID].ToString(0), Equals, tests[i].hist) + if test.idxCols == 0 { + require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) } else { - c.Assert(tbl.Indices[t.rangeID].ToString(1), Equals, tests[i].hist) + require.Equal(t, tests[i].hist, tbl.Indices[test.rangeID].ToString(1)) } - val, err := codec.EncodeKey(testKit.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - c.Assert(err, IsNil) - c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, uint64(t.eqCount)) + val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) + require.NoError(t, err) + require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) } } -func (s *testStatsSuite) TestIndexQueryFeedback4TopN(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestIndexQueryFeedback4TopN(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() @@ -1755,37 +1612,38 @@ func (s *testStatsSuite) TestIndexQueryFeedback4TopN(c *C) { for i := 0; i < 20; i++ { testKit.MustExec(`insert into t values (1)`) } - h := s.do.StatsHandle() + h := dom.StatsHandle() err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("set @@session.tidb_analyze_version = 1") testKit.MustExec("set @@tidb_enable_fast_analyze = 1") testKit.MustExec("analyze table t with 3 buckets") for i := 0; i < 20; i++ { testKit.MustExec(`insert into t values (1)`) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - is := s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + is := dom.InfoSchema() + require.NoError(t, h.Update(is)) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() testKit.MustQuery("select * from t use index(idx) where a = 1") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, h.Update(is)) tbl := h.GetTableStats(tblInfo) - val, err := codec.EncodeKey(testKit.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - c.Assert(err, IsNil) - c.Assert(tbl.Indices[1].CMSketch.QueryBytes(val), Equals, uint64(40)) + val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) + require.NoError(t, err) + require.Equal(t, uint64(40), tbl.Indices[1].CMSketch.QueryBytes(val)) } -func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestAbnormalIndexFeedback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() @@ -1808,11 +1666,11 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { testKit.MustExec("analyze table t with 3 buckets, 0 topn") testKit.MustExec("delete from t where a = 1") testKit.MustExec("delete from t where b > 10") - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tblInfo := table.Meta() - h := s.do.StatsHandle() + h := dom.StatsHandle() tests := []struct { sql string hist string @@ -1843,24 +1701,25 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { eqCount: 3, }, } - for i, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(h.Update(is), IsNil) + for i, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, h.Update(is)) tbl := h.GetTableStats(tblInfo) - c.Assert(tbl.Columns[t.rangeID].ToString(0), Equals, tests[i].hist) - val, err := codec.EncodeKey(testKit.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - c.Assert(err, IsNil) - c.Assert(tbl.Indices[t.idxID].CMSketch.QueryBytes(val), Equals, uint64(t.eqCount)) + require.Equal(t, tests[i].hist, tbl.Columns[test.rangeID].ToString(0)) + val, err := codec.EncodeKey(testKit.Session().GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) + require.NoError(t, err) + require.Equal(t, uint64(test.eqCount), tbl.Indices[test.idxID].CMSketch.QueryBytes(val)) } } -func (s *testStatsSuite) TestFeedbackRanges(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - h := s.do.StatsHandle() +func TestFeedbackRanges(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() oriProbability := statistics.FeedbackProbability.Load() oriNumber := statistics.MaxNumberOfRanges oriMinLogCount := handle.MinLogScanCount.Load() @@ -1881,14 +1740,14 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) tests := []struct { sql string hist string @@ -1919,25 +1778,26 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { colID: 2, }, } - is := s.do.InfoSchema() + is := dom.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - for i, t := range tests { - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + for _, test := range tests { + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, err) + require.NoError(t, h.Update(is)) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) - c.Assert(tbl.Columns[t.colID].ToString(0), Equals, tests[i].hist) + require.Equal(t, test.hist, tbl.Columns[test.colID].ToString(0)) } } -func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - h := s.do.StatsHandle() +func TestUnsignedFeedbackRanges(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() oriProbability := statistics.FeedbackProbability.Load() oriMinLogCount := handle.MinLogScanCount.Load() @@ -1962,16 +1822,16 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) } err := h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) + require.NoError(t, err) err = h.HandleDDLEvent(<-h.DDLEventCh()) - c.Assert(err, IsNil) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, err) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t, t1 with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d)", i)) testKit.MustExec(fmt.Sprintf("insert into t1 values (%d)", i)) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) tests := []struct { sql string hist string @@ -2010,49 +1870,51 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { tblName: "t1", }, } - is := s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) - for i, t := range tests { - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(t.tblName)) - c.Assert(err, IsNil) - testKit.MustQuery(t.sql) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.DumpStatsFeedbackToKV(), IsNil) - c.Assert(h.HandleUpdateStats(s.do.InfoSchema()), IsNil) - c.Assert(err, IsNil) - c.Assert(h.Update(is), IsNil) + is := dom.InfoSchema() + require.NoError(t, h.Update(is)) + for _, test := range tests { + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr(test.tblName)) + require.NoError(t, err) + testKit.MustQuery(test.sql) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.DumpStatsFeedbackToKV()) + require.NoError(t, h.HandleUpdateStats(dom.InfoSchema())) + require.NoError(t, err) + require.NoError(t, h.Update(is)) tblInfo := table.Meta() tbl := h.GetTableStats(tblInfo) - c.Assert(tbl.Columns[1].ToString(0), Equals, tests[i].hist) + require.Equal(t, test.hist, tbl.Columns[1].ToString(0)) } } -func (s *testStatsSuite) TestLoadHistCorrelation(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) - h := s.do.StatsHandle() +func TestLoadHistCorrelation(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + h := dom.StatsHandle() origLease := h.Lease() h.SetLease(time.Second) defer func() { h.SetLease(origLease) }() testKit.MustExec("use test") testKit.MustExec("create table t(c int)") testKit.MustExec("insert into t values(1),(2),(3),(4),(5)") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t") h.Clear() - c.Assert(h.Update(s.do.InfoSchema()), IsNil) + require.NoError(t, h.Update(dom.InfoSchema())) result := testKit.MustQuery("show stats_histograms where Table_name = 't'") - c.Assert(len(result.Rows()), Equals, 0) + require.Len(t, result.Rows(), 0) testKit.MustExec("explain select * from t where c = 1") - c.Assert(h.LoadNeededHistograms(), IsNil) + require.NoError(t, h.LoadNeededHistograms()) result = testKit.MustQuery("show stats_histograms where Table_name = 't'") - c.Assert(len(result.Rows()), Equals, 1) - c.Assert(result.Rows()[0][9], Equals, "1") + require.Len(t, result.Rows(), 1) + require.Equal(t, "1", result.Rows()[0][9]) } -func (s *testStatsSuite) TestDeleteUpdateFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestDeleteUpdateFeedback(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() defer func() { @@ -2060,33 +1922,34 @@ func (s *testStatsSuite) TestDeleteUpdateFeedback(c *C) { }() statistics.FeedbackProbability.Store(1) - h := s.do.StatsHandle() + h := dom.StatsHandle() testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) } - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("analyze table t with 3 buckets") testKit.MustExec("delete from t where a = 1") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.GetQueryFeedback().Size, Equals, 0) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Equal(t, 0, h.GetQueryFeedback().Size) testKit.MustExec("update t set a = 6 where a = 2") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.GetQueryFeedback().Size, Equals, 0) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Equal(t, 0, h.GetQueryFeedback().Size) testKit.MustExec("explain analyze delete from t where a = 3") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.GetQueryFeedback().Size, Equals, 0) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.Equal(t, 0, h.GetQueryFeedback().Size) } -func (s *testStatsSuite) BenchmarkHandleAutoAnalyze(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func BenchmarkHandleAutoAnalyze(b *testing.B) { + store, dom, clean := testkit.CreateMockStoreAndDomain(b) + defer clean() + testKit := testkit.NewTestKit(b, store) testKit.MustExec("use test") - h := s.do.StatsHandle() - is := s.do.InfoSchema() - for i := 0; i < c.N; i++ { + h := dom.StatsHandle() + is := dom.InfoSchema() + for i := 0; i < b.N; i++ { h.HandleAutoAnalyze(is) } } @@ -2103,9 +1966,10 @@ func subtraction(newMetric *dto.Metric, oldMetric *dto.Metric) int { return newNum - oldNum } -func (s *testStatsSuite) TestDisableFeedback(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestDisableFeedback(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() defer func() { @@ -2114,7 +1978,7 @@ func (s *testStatsSuite) TestDisableFeedback(c *C) { statistics.FeedbackProbability.Store(0.0) oldNum := &dto.Metric{} err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - c.Assert(err, IsNil) + require.NoError(t, err) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx_a(a))") testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") @@ -2125,13 +1989,14 @@ func (s *testStatsSuite) TestDisableFeedback(c *C) { newNum := &dto.Metric{} err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - c.Assert(err, IsNil) - c.Assert(subtraction(newNum, oldNum), Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, subtraction(newNum, oldNum)) } -func (s *testStatsSuite) TestFeedbackCounter(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestFeedbackCounter(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) oriProbability := statistics.FeedbackProbability.Load() defer func() { @@ -2140,7 +2005,7 @@ func (s *testStatsSuite) TestFeedbackCounter(c *C) { statistics.FeedbackProbability.Store(1) oldNum := &dto.Metric{} err := metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(oldNum) - c.Assert(err, IsNil) + require.NoError(t, err) testKit.MustExec("use test") testKit.MustExec("create table t (a int, b int, index idx_a(a))") testKit.MustExec("insert into t values (1, 1), (2, 2), (3, 3), (5, 5)") @@ -2151,11 +2016,11 @@ func (s *testStatsSuite) TestFeedbackCounter(c *C) { newNum := &dto.Metric{} err = metrics.StoreQueryFeedbackCounter.WithLabelValues(metrics.LblOK).Write(newNum) - c.Assert(err, IsNil) - c.Assert(subtraction(newNum, oldNum), Equals, 20) + require.NoError(t, err) + require.Equal(t, 20, subtraction(newNum, oldNum)) } -func (s *testSerialStatsSuite) TestMergeTopN(c *C) { +func TestMergeTopN(t *testing.T) { // Move this test to here to avoid race test. tests := []struct { topnNum int @@ -2188,13 +2053,13 @@ func (s *testSerialStatsSuite) TestMergeTopN(c *C) { maxTopNCnt: 100, }, } - for _, t := range tests { - topnNum, n := t.topnNum, t.n - maxTopNVal, maxTopNCnt := t.maxTopNVal, t.maxTopNCnt + for _, test := range tests { + topnNum, n := test.topnNum, test.n + maxTopNVal, maxTopNCnt := test.maxTopNVal, test.maxTopNCnt // the number of maxTopNVal should be bigger than n. ok := maxTopNVal >= n - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) topNs := make([]*statistics.TopN, 0, topnNum) res := make(map[int]uint64) @@ -2224,28 +2089,29 @@ func (s *testSerialStatsSuite) TestMergeTopN(c *C) { var minTopNCnt uint64 for _, topNMeta := range topN.TopN { val, err := strconv.Atoi(string(topNMeta.Encoded)) - c.Assert(err, IsNil) - c.Assert(topNMeta.Count, Equals, res[val]) + require.NoError(t, err) + require.Equal(t, res[val], topNMeta.Count) minTopNCnt = topNMeta.Count } if remainTopN != nil { cnt += len(remainTopN) for _, remainTopNMeta := range remainTopN { val, err := strconv.Atoi(string(remainTopNMeta.Encoded)) - c.Assert(err, IsNil) - c.Assert(remainTopNMeta.Count, Equals, res[val]) + require.NoError(t, err) + require.Equal(t, res[val], remainTopNMeta.Count) // The count of value in remainTopN may equal to the min count of value in TopN. ok = minTopNCnt >= remainTopNMeta.Count - c.Assert(ok, Equals, true) + require.Equal(t, true, ok) } } - c.Assert(cnt, Equals, len(res)) + require.Equal(t, len(res), cnt) } } -func (s *testSerialStatsSuite) TestAutoUpdatePartitionInDynamicOnlyMode(c *C) { - defer cleanEnv(c, s.store, s.do) - testKit := testkit.NewTestKit(c, s.store) +func TestAutoUpdatePartitionInDynamicOnlyMode(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testkit.WithPruneMode(testKit, variable.DynamicOnly, func() { testKit.MustExec("use test") testKit.MustExec("set @@tidb_analyze_version = 2;") @@ -2256,14 +2122,14 @@ func (s *testSerialStatsSuite) TestAutoUpdatePartitionInDynamicOnlyMode(c *C) { partition p1 values less than (20), partition p2 values less than (30))`) - do := s.do + do := dom is := do.InfoSchema() h := do.StatsHandle() - c.Assert(h.RefreshVars(), IsNil) - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.RefreshVars()) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) testKit.MustExec("insert into t values (1, 'a'), (2, 'b'), (11, 'c'), (12, 'd'), (21, 'e'), (22, 'f')") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) testKit.MustExec("set @@tidb_analyze_version = 2") testKit.MustExec("analyze table t") @@ -2274,42 +2140,43 @@ func (s *testSerialStatsSuite) TestAutoUpdatePartitionInDynamicOnlyMode(c *C) { testKit.MustExec("set global tidb_auto_analyze_ratio = 0.0") }() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) - c.Assert(err, IsNil) + require.NoError(t, err) tableInfo := tbl.Meta() pi := tableInfo.GetPartitionInfo() globalStats := h.GetTableStats(tableInfo) partitionStats := h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) - c.Assert(globalStats.Count, Equals, int64(6)) - c.Assert(globalStats.ModifyCount, Equals, int64(0)) - c.Assert(partitionStats.Count, Equals, int64(2)) - c.Assert(partitionStats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(6), globalStats.Count) + require.Equal(t, int64(0), globalStats.ModifyCount) + require.Equal(t, int64(2), partitionStats.Count) + require.Equal(t, int64(0), partitionStats.ModifyCount) testKit.MustExec("insert into t values (3, 'g')") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) globalStats = h.GetTableStats(tableInfo) partitionStats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) - c.Assert(globalStats.Count, Equals, int64(7)) - c.Assert(globalStats.ModifyCount, Equals, int64(1)) - c.Assert(partitionStats.Count, Equals, int64(3)) - c.Assert(partitionStats.ModifyCount, Equals, int64(1)) + require.Equal(t, int64(7), globalStats.Count) + require.Equal(t, int64(1), globalStats.ModifyCount) + require.Equal(t, int64(3), partitionStats.Count) + require.Equal(t, int64(1), partitionStats.ModifyCount) h.HandleAutoAnalyze(is) - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.Update(is)) globalStats = h.GetTableStats(tableInfo) partitionStats = h.GetPartitionStats(tableInfo, pi.Definitions[0].ID) - c.Assert(globalStats.Count, Equals, int64(7)) - c.Assert(globalStats.ModifyCount, Equals, int64(0)) - c.Assert(partitionStats.Count, Equals, int64(3)) - c.Assert(partitionStats.ModifyCount, Equals, int64(0)) + require.Equal(t, int64(7), globalStats.Count) + require.Equal(t, int64(0), globalStats.ModifyCount) + require.Equal(t, int64(3), partitionStats.Count) + require.Equal(t, int64(0), partitionStats.ModifyCount) }) } -func (s *testSerialStatsSuite) TestAutoAnalyzeRatio(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestAutoAnalyzeRatio(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) oriStart := tk.MustQuery("select @@tidb_auto_analyze_start_time").Rows()[0][0].(string) oriEnd := tk.MustQuery("select @@tidb_auto_analyze_end_time").Rows()[0][0].(string) @@ -2320,40 +2187,41 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeRatio(c *C) { tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='%v'", oriEnd)) }() - h := s.do.StatsHandle() + h := dom.StatsHandle() tk.MustExec("use test") tk.MustExec("create table t (a int)") - c.Assert(h.HandleDDLEvent(<-h.DDLEventCh()), IsNil) + require.NoError(t, h.HandleDDLEvent(<-h.DDLEventCh())) tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", 19)) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - is := s.do.InfoSchema() - c.Assert(h.Update(is), IsNil) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + is := dom.InfoSchema() + require.NoError(t, h.Update(is)) // To pass the stats.Pseudo check in autoAnalyzeTable tk.MustExec("analyze table t") tk.MustExec("explain select * from t where a = 1") - c.Assert(h.LoadNeededHistograms(), IsNil) + require.NoError(t, h.LoadNeededHistograms()) tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") tk.MustExec("insert into t values (1)" + strings.Repeat(", (1)", 10)) - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - c.Assert(h.HandleAutoAnalyze(is), IsTrue) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) + require.True(t, h.HandleAutoAnalyze(is)) tk.MustExec("delete from t limit 12") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - c.Assert(h.HandleAutoAnalyze(is), IsFalse) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) + require.False(t, h.HandleAutoAnalyze(is)) tk.MustExec("delete from t limit 4") - c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) - c.Assert(h.Update(is), IsNil) - c.Assert(h.HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) + require.NoError(t, h.DumpStatsDeltaToKV(handle.DumpAll)) + require.NoError(t, h.Update(is)) + require.True(t, h.HandleAutoAnalyze(dom.InfoSchema())) } -func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) +func TestDumpColumnStatsUsage(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) originalVal := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { @@ -2361,7 +2229,7 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { }() tk.MustExec("set global tidb_enable_column_tracking = 1") - h := s.do.StatsHandle() + h := dom.StatsHandle() tk.MustExec("use test") tk.MustExec("create table t1(a int, b int)") tk.MustExec("create table t2(a int, b int)") @@ -2371,32 +2239,32 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { tk.MustExec("insert into t3 values (1, 2), (3, 4), (11, 12), (13, 14)") tk.MustExec("select * from t1 where a > 1") tk.MustExec("select * from t2 where b < 10") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) // t1.a is collected as predicate column rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t1", "", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't2'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t2", "", "b"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t2", "", "b"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") tk.MustExec("analyze table t1") tk.MustExec("select * from t1 where b > 1") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) // t1.a updates last_used_at first and then updates last_analyzed_at while t1.b updates last_analyzed_at first and then updates last_used_at. // Check both of them behave as expected. rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) != "", IsTrue) - c.Assert(rows[1][:4], DeepEquals, []interface{}{"test", "t1", "", "b"}) - c.Assert(rows[1][4].(string) != "", IsTrue) - c.Assert(rows[1][5].(string) != "", IsTrue) + require.Len(t, rows, 2) + require.Equal(t, []interface{}{"test", "t1", "", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) != "") + require.Equal(t, []interface{}{"test", "t1", "", "b"}, rows[1][:4]) + require.True(t, rows[1][4].(string) != "") + require.True(t, rows[1][5].(string) != "") // Test partition table. // No matter whether it is static or dynamic pruning mode, we record predicate columns using table ID rather than partition ID. @@ -2404,12 +2272,12 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", val)) tk.MustExec("delete from mysql.column_stats_usage") tk.MustExec("select * from t3 where a < 5") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't3'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t3", "global", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t3", "global", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") } // Test non-correlated subquery. @@ -2417,20 +2285,20 @@ func (s *testSerialStatsSuite) TestDumpColumnStatsUsage(c *C) { // Hence we put the test of collecting predicate columns for non-correlated subquery here. tk.MustExec("delete from mysql.column_stats_usage") tk.MustExec("select * from t2 where t2.a > (select count(*) from t1 where t1.b > 1)") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "b"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t1", "", "b"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't2'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t2", "", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t2", "", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") } -func (s *testSerialStatsSuite) TestCollectPredicateColumnsFromExecute(c *C) { +func TestCollectPredicateColumnsFromExecute(t *testing.T) { for _, val := range []bool{false, true} { func(planCache bool) { originalVal1 := plannercore.PreparedPlanCacheEnabled() @@ -2439,8 +2307,9 @@ func (s *testSerialStatsSuite) TestCollectPredicateColumnsFromExecute(c *C) { }() plannercore.SetPreparedPlanCache(planCache) - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) originalVal2 := tk.MustQuery("select @@tidb_enable_column_tracking").Rows()[0][0].(string) defer func() { @@ -2448,47 +2317,48 @@ func (s *testSerialStatsSuite) TestCollectPredicateColumnsFromExecute(c *C) { }() tk.MustExec("set global tidb_enable_column_tracking = 1") - h := s.do.StatsHandle() + h := dom.StatsHandle() tk.MustExec("use test") tk.MustExec("create table t1(a int, b int)") tk.MustExec("prepare stmt from 'select * from t1 where a > ?'") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) // Prepare only converts sql string to ast and doesn't do optimization, so no predicate column is collected. tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Check(testkit.Rows()) tk.MustExec("set @p1 = 1") tk.MustExec("execute stmt using @p1") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t1", "", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") tk.MustExec("delete from mysql.column_stats_usage") tk.MustExec("set @p2 = 2") tk.MustExec("execute stmt using @p2") if planCache { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) // If the second execution uses the cached plan, no predicate column is collected. tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Check(testkit.Rows()) } else { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't1'").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][:4], DeepEquals, []interface{}{"test", "t1", "", "a"}) - c.Assert(rows[0][4].(string) != "", IsTrue) - c.Assert(rows[0][5].(string) == "", IsTrue) + require.Len(t, rows, 1) + require.Equal(t, []interface{}{"test", "t1", "", "a"}, rows[0][:4]) + require.True(t, rows[0][4].(string) != "") + require.True(t, rows[0][5].(string) == "") } }(val) } } -func (s *testSerialStatsSuite) TestEnableAndDisableColumnTracking(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - h := s.do.StatsHandle() +func TestEnableAndDisableColumnTracking(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + h := dom.StatsHandle() tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, c int)") @@ -2500,10 +2370,10 @@ func (s *testSerialStatsSuite) TestEnableAndDisableColumnTracking(c *C) { tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where b > 1") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows := tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Rows() - c.Assert(len(rows), Equals, 1) - c.Assert(rows[0][3], Equals, "b") + require.Len(t, rows, 1) + require.Equal(t, "b", rows[0][3]) tk.MustExec("set global tidb_enable_column_tracking = 0") // After tidb_enable_column_tracking is set to 0, the predicate columns collected before are invalidated. @@ -2512,17 +2382,17 @@ func (s *testSerialStatsSuite) TestEnableAndDisableColumnTracking(c *C) { // Sleep for 1.5s to let `last_used_at` be larger than `tidb_disable_tracking_time`. time.Sleep(1500 * time.Millisecond) tk.MustExec("select * from t where a > 1") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) // We don't collect predicate columns when tidb_enable_column_tracking = 0 tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Check(testkit.Rows()) tk.MustExec("set global tidb_enable_column_tracking = 1") tk.MustExec("select * from t where b < 1 and c > 1") - c.Assert(h.DumpColStatsUsageToKV(), IsNil) + require.NoError(t, h.DumpColStatsUsageToKV()) rows = tk.MustQuery("show column_stats_usage where db_name = 'test' and table_name = 't' and last_used_at is not null").Sort().Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[0][3], Equals, "b") - c.Assert(rows[1][3], Equals, "c") + require.Len(t, rows, 2) + require.Equal(t, "b", rows[0][3]) + require.Equal(t, "c", rows[1][3]) // Test invalidating predicate columns again in order to check that tidb_disable_tracking_time can be updated. tk.MustExec("set global tidb_enable_column_tracking = 0") diff --git a/statistics/main_test.go b/statistics/main_test.go index 4b0960f69a735..e03ea212f291e 100644 --- a/statistics/main_test.go +++ b/statistics/main_test.go @@ -48,7 +48,7 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "trace_suite") opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/statistics/scalar.go b/statistics/scalar.go index f4be9f4647aa0..82a07e1be840b 100644 --- a/statistics/scalar.go +++ b/statistics/scalar.go @@ -96,22 +96,22 @@ func convertDatumToScalar(value *types.Datum, commonPfxLen int) float64 { // of lower and upper equals to the common prefix of the lower, upper and the value. For some simple types like `Int64`, // we do not convert it because we can directly infer the scalar value. func (hg *Histogram) PreCalculateScalar() { - len := hg.Len() - if len == 0 { + l := hg.Len() + if l == 0 { return } switch hg.GetLower(0).Kind() { case types.KindMysqlDecimal, types.KindMysqlTime: - hg.scalars = make([]scalar, len) - for i := 0; i < len; i++ { + hg.scalars = make([]scalar, l) + for i := 0; i < l; i++ { hg.scalars[i] = scalar{ lower: convertDatumToScalar(hg.GetLower(i), 0), upper: convertDatumToScalar(hg.GetUpper(i), 0), } } case types.KindBytes, types.KindString: - hg.scalars = make([]scalar, len) - for i := 0; i < len; i++ { + hg.scalars = make([]scalar, l) + for i := 0; i < l; i++ { lower, upper := hg.GetLower(i), hg.GetUpper(i) common := commonPrefixLength(lower.GetBytes(), upper.GetBytes()) hg.scalars[i] = scalar{ @@ -252,8 +252,8 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type return values case types.KindMysqlDuration: lowDur, highDur := low.GetMysqlDuration(), high.GetMysqlDuration() - fsp := mathutil.MaxInt8(lowDur.Fsp, highDur.Fsp) - stepSize := int64(math.Pow10(int(types.MaxFsp-fsp))) * int64(time.Microsecond) + fsp := mathutil.Max(lowDur.Fsp, highDur.Fsp) + stepSize := int64(math.Pow10(types.MaxFsp-fsp)) * int64(time.Microsecond) lowDur.Duration = lowDur.Duration.Round(time.Duration(stepSize)) remaining := int64(highDur.Duration-lowDur.Duration)/stepSize + 1 - int64(exclude) if remaining <= 0 || remaining >= maxNumStep { @@ -273,7 +273,7 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type if lowTime.Type() != highTime.Type() { return nil } - fsp := mathutil.MaxInt8(lowTime.Fsp(), highTime.Fsp()) + fsp := mathutil.Max(lowTime.Fsp(), highTime.Fsp()) var stepSize int64 sc := &stmtctx.StatementContext{TimeZone: time.UTC} if lowTime.Type() == mysql.TypeDate { @@ -285,7 +285,7 @@ func enumRangeValues(low, high types.Datum, lowExclude, highExclude bool) []type if err != nil { return nil } - stepSize = int64(math.Pow10(int(types.MaxFsp-fsp))) * int64(time.Microsecond) + stepSize = int64(math.Pow10(types.MaxFsp-fsp)) * int64(time.Microsecond) } remaining := int64(highTime.Sub(sc, &lowTime).Duration)/stepSize + 1 - int64(exclude) // When `highTime` is much larger than `lowTime`, `remaining` may be overflowed to a negative value. diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index c96781db61b51..3ad5dfbd33d0b 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -49,8 +49,6 @@ func TestCollationColumnEstimate(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a varchar(20) collate utf8mb4_general_ci)") @@ -710,11 +708,11 @@ func TestIndexEstimationCrossValidate(t *testing.T) { tk.MustExec("create table t(a int, b int, key(a,b))") tk.MustExec("insert into t values(1, 1), (1, 2), (1, 3), (2, 2)") tk.MustExec("analyze table t") - require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/statistics/table/mockQueryBytesMaxUint64", `return(100000)`)) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/table/mockQueryBytesMaxUint64", `return(100000)`)) tk.MustQuery("explain select * from t where a = 1 and b = 2").Check(testkit.Rows( "IndexReader_6 1.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 1.00 cop[tikv] table:t, index:a(a, b) range:[1 2,1 2], keep order:false")) - require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/statistics/table/mockQueryBytesMaxUint64")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/statistics/table/mockQueryBytesMaxUint64")) // Test issue 22466 tk.MustExec("drop table if exists t2") diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index cf753ca21b4f0..0c93cabcff7b6 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -200,7 +200,7 @@ func TestPseudoTable(t *testing.T) { } ti.Columns = append(ti.Columns, colInfo) tbl := PseudoTable(ti) - require.Equal(t, len(tbl.Columns), 1) + require.Len(t, tbl.Columns, 1) require.Greater(t, tbl.Count, int64(0)) sctx := mock.NewContext() count := tbl.ColumnLessRowCount(sctx, types.NewIntDatum(100), colInfo.ID) diff --git a/store/batch_coprocessor_test.go b/store/batch_coprocessor_test.go index 07345156bb389..51097f1deb12d 100644 --- a/store/batch_coprocessor_test.go +++ b/store/batch_coprocessor_test.go @@ -23,11 +23,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/testutils" @@ -52,16 +49,6 @@ func createMockTiKVStoreOptions(tiflashNum int) []mockstore.MockTiKVStoreOption } } -func testGetTableByName(t *testing.T, ctx sessionctx.Context, db, table string) table.Table { - dom := domain.GetDomain(ctx) - // Make sure the table schema is the new schema. - err := dom.Reload() - require.NoError(t, err) - tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) - require.NoError(t, err) - return tbl -} - func TestStoreErr(t *testing.T) { store, clean := testkit.CreateMockStore(t, createMockTiKVStoreOptions(1)...) defer clean() @@ -75,7 +62,7 @@ func TestStoreErr(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t(a int not null, b int not null)") tk.MustExec("alter table t set tiflash replica 1") - tb := testGetTableByName(t, tk.Session(), "test", "t") + tb := testkit.TestGetTableByName(t, tk.Session(), "test", "t") err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) require.NoError(t, err) @@ -111,7 +98,7 @@ func TestStoreSwitchPeer(t *testing.T) { tk.MustExec("use test") tk.MustExec("create table t(a int not null, b int not null)") tk.MustExec("alter table t set tiflash replica 1") - tb := testGetTableByName(t, tk.Session(), "test", "t") + tb := testkit.TestGetTableByName(t, tk.Session(), "test", "t") err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) require.NoError(t, err) diff --git a/store/driver/error/error.go b/store/driver/error/error.go index 8260acd2640bb..6b7b444239d9c 100644 --- a/store/driver/error/error.go +++ b/store/driver/error/error.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package error +package error //nolint:predeclared import ( stderrs "errors" diff --git a/store/driver/error/error_test.go b/store/driver/error/error_test.go index 6d2222b29bc59..2c5def15ed577 100644 --- a/store/driver/error/error_test.go +++ b/store/driver/error/error_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package error +package error //nolint:predeclared import ( "testing" diff --git a/store/driver/main_test.go b/store/driver/main_test.go index 7e9de1553e8aa..71e63367803ab 100644 --- a/store/driver/main_test.go +++ b/store/driver/main_test.go @@ -40,7 +40,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 730dba0c3c7fb..373d72c66b23b 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -237,6 +237,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.SetCommitTSUpperBoundCheck(val.(func(commitTS uint64) bool)) case kv.RPCInterceptor: txn.KVTxn.SetRPCInterceptor(val.(interceptor.RPCInterceptor)) + case kv.AssertionLevel: + txn.KVTxn.SetAssertionLevel(val.(kvrpcpb.AssertionLevel)) } } @@ -290,6 +292,19 @@ func (txn *tikvTxn) extractKeyExistsErr(key kv.Key) error { return extractKeyExistsErrFromIndex(key, value, tblInfo, indexID) } +// SetAssertion sets an assertion for the key operation. +func (txn *tikvTxn) SetAssertion(key []byte, assertion ...kv.FlagsOp) error { + f, err := txn.GetUnionStore().GetMemBuffer().GetFlags(key) + if err != nil && !tikverr.IsErrNotFound(err) { + return err + } + if err == nil && f.HasAssertionFlags() { + return nil + } + txn.GetUnionStore().GetMemBuffer().UpdateFlags(key, getTiKVFlagsOps(assertion)...) + return nil +} + // TiDBKVFilter is the filter specific to TiDB to filter out KV pairs that needn't be committed. type TiDBKVFilter struct{} diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index fc7b97dc74c07..6b2dcc283213e 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -146,6 +146,14 @@ func getTiDBKeyFlags(flag tikvstore.KeyFlags) kv.KeyFlags { if flag.HasNeedLocked() { v = kv.ApplyFlagsOps(v, kv.SetNeedLocked) } + + if flag.HasAssertExist() { + v = kv.ApplyFlagsOps(v, kv.SetAssertExist) + } else if flag.HasAssertNotExist() { + v = kv.ApplyFlagsOps(v, kv.SetAssertNotExist) + } else if flag.HasAssertUnknown() { + v = kv.ApplyFlagsOps(v, kv.SetAssertUnknown) + } return v } @@ -155,6 +163,14 @@ func getTiKVFlagsOp(op kv.FlagsOp) tikvstore.FlagsOp { return tikvstore.SetPresumeKeyNotExists case kv.SetNeedLocked: return tikvstore.SetNeedLocked + case kv.SetAssertExist: + return tikvstore.SetAssertExist + case kv.SetAssertNotExist: + return tikvstore.SetAssertNotExist + case kv.SetAssertUnknown: + return tikvstore.SetAssertUnknown + case kv.SetAssertNone: + return tikvstore.SetAssertNone } return 0 } diff --git a/store/gcworker/main_test.go b/store/gcworker/main_test.go index 89909d438672a..400d97e9a0bf4 100644 --- a/store/gcworker/main_test.go +++ b/store/gcworker/main_test.go @@ -28,7 +28,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() tikv.EnableFailpoints() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } callback := func(i int) int { diff --git a/store/helper/helper.go b/store/helper/helper.go index dc482a2a6d4eb..0fd44304d59f1 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -772,10 +772,17 @@ func (h *Helper) GetRegionsInfo() (*RegionsInfo, error) { return ®ionsInfo, err } +// GetStoreRegionsInfo gets the region in given store. +func (h *Helper) GetStoreRegionsInfo(storeID uint64) (*RegionsInfo, error) { + var regionsInfo RegionsInfo + err := h.requestPD("GET", pdapi.StoreRegions+"/"+strconv.FormatUint(storeID, 10), nil, ®ionsInfo) + return ®ionsInfo, err +} + // GetRegionInfoByID gets the region information of the region ID by using PD's api. func (h *Helper) GetRegionInfoByID(regionID uint64) (*RegionInfo, error) { var regionInfo RegionInfo - err := h.requestPD("GET", pdapi.RegionByID+strconv.FormatUint(regionID, 10), nil, ®ionInfo) + err := h.requestPD("GET", pdapi.RegionByID+"/"+strconv.FormatUint(regionID, 10), nil, ®ionInfo) return ®ionInfo, err } @@ -1132,30 +1139,42 @@ func GetTiFlashTableIDFromEndKey(endKey string) int64 { // ComputeTiFlashStatus is helper function for CollectTiFlashStatus. func ComputeTiFlashStatus(reader *bufio.Reader, regionReplica *map[int64]int) error { - ns, _, _ := reader.ReadLine() - n, err := strconv.ParseInt(string(ns), 10, 64) + ns, err := reader.ReadString('\n') if err != nil { return errors.Trace(err) } - for i := int64(0); i < n; i++ { - rs, _, _ := reader.ReadLine() - srs := strings.Trim(string(rs), "\r\n\t") - splits := strings.Split(srs, " ") - for _, s := range splits { - // For (`table`, `store`), has region `r` - if s == "" { - continue - } - r, err := strconv.ParseInt(s, 10, 32) - if err != nil { - return errors.Trace(err) - } - if c, ok := (*regionReplica)[r]; ok { - (*regionReplica)[r] = c + 1 - } else { - (*regionReplica)[r] = 1 - } + // The count + ns = strings.Trim(ns, "\r\n\t") + n, err := strconv.ParseInt(ns, 10, 64) + if err != nil { + return errors.Trace(err) + } + // The regions + regions, err := reader.ReadString('\n') + if err != nil { + return errors.Trace(err) + } + regions = strings.Trim(regions, "\r\n\t") + splits := strings.Split(regions, " ") + realN := int64(0) + for _, s := range splits { + // For (`table`, `store`), has region `r` + if s == "" { + continue } + realN += 1 + r, err := strconv.ParseInt(s, 10, 32) + if err != nil { + return errors.Trace(err) + } + if c, ok := (*regionReplica)[r]; ok { + (*regionReplica)[r] = c + 1 + } else { + (*regionReplica)[r] = 1 + } + } + if n != realN { + logutil.BgLogger().Warn("ComputeTiFlashStatus count check failed", zap.Int64("claim", n), zap.Int64("real", realN)) } return nil } diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index 37e1359e975c5..acd9bac0e5290 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -18,6 +18,7 @@ import ( "bufio" "crypto/tls" "encoding/json" + "fmt" "net/http" "net/http/httptest" "strings" @@ -437,11 +438,9 @@ func mockStoreStatResponse(w http.ResponseWriter, _ *http.Request) { func TestComputeTiFlashStatus(t *testing.T) { regionReplica := make(map[int64]int) // There are no region in this TiFlash store. - resp1 := "0\n\n" - // There are one region 1009 in this TiFlash store. - resp2 := "1\n1009 1010 \n" - br1 := bufio.NewReader(strings.NewReader(resp1)) - br2 := bufio.NewReader(strings.NewReader(resp2)) + br1 := bufio.NewReader(strings.NewReader("0\n\n")) + // There are 2 regions 1009/1010 in this TiFlash store. + br2 := bufio.NewReader(strings.NewReader("2\n1009 1010 \n")) err := helper.ComputeTiFlashStatus(br1, ®ionReplica) require.NoError(t, err) err = helper.ComputeTiFlashStatus(br2, ®ionReplica) @@ -453,6 +452,19 @@ func TestComputeTiFlashStatus(t *testing.T) { v, ok = regionReplica[1010] require.Equal(t, v, 1) require.Equal(t, ok, true) + + regionReplica2 := make(map[int64]int) + var sb strings.Builder + for i := 1000; i < 3000; i++ { + sb.WriteString(fmt.Sprintf("%v ", i)) + } + s := fmt.Sprintf("2000\n%v\n", sb.String()) + require.NoError(t, helper.ComputeTiFlashStatus(bufio.NewReader(strings.NewReader(s)), ®ionReplica2)) + require.Equal(t, 2000, len(regionReplica2)) + for i := 1000; i < 3000; i++ { + _, ok := regionReplica2[int64(i)] + require.True(t, ok) + } } // TestTableRange tests the first part of GetPDRegionStats. diff --git a/store/main_test.go b/store/main_test.go index b76d49ae6800c..3dccbd0df88b0 100644 --- a/store/main_test.go +++ b/store/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/store/mockstore/mockcopr/main_test.go b/store/mockstore/mockcopr/main_test.go index 91d38a4cff79c..74b84ce7b0475 100644 --- a/store/mockstore/mockcopr/main_test.go +++ b/store/mockstore/mockcopr/main_test.go @@ -26,7 +26,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } callback := func(i int) int { diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index efc845820d260..d1dee3b888538 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/mockstore/unistore/lockstore" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/dbreader" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc" "github.com/pingcap/tidb/tablecodec" @@ -604,7 +605,7 @@ func (e *closureExecutor) isPointGetRange(ran kv.KeyRange) bool { func (e *closureExecutor) checkRangeLock() error { if !e.ignoreLock && !e.lockChecked { for _, ran := range e.kvRanges { - err := e.checkRangeLockForRange(ran) + err := checkRangeLockForRange(e.lockStore, e.startTS, e.resolvedLocks, ran) if err != nil { return err } @@ -614,14 +615,14 @@ func (e *closureExecutor) checkRangeLock() error { return nil } -func (e *closureExecutor) checkRangeLockForRange(ran kv.KeyRange) error { - it := e.lockStore.NewIterator() +func checkRangeLockForRange(store *lockstore.MemStore, startTS uint64, resolvedLocks []uint64, ran kv.KeyRange) error { + it := store.NewIterator() for it.Seek(ran.StartKey); it.Valid(); it.Next() { if exceedEndKey(it.Key(), ran.EndKey) { break } lock := mvcc.DecodeLock(it.Value()) - err := checkLock(lock, it.Key(), e.startTS, e.resolvedLocks) + err := checkLock(lock, it.Key(), startTS, resolvedLocks) if err != nil { return err } diff --git a/store/mockstore/unistore/cophandler/cop_handler.go b/store/mockstore/unistore/cophandler/cop_handler.go index fa7b784126e3e..b39baccda1125 100644 --- a/store/mockstore/unistore/cophandler/cop_handler.go +++ b/store/mockstore/unistore/cophandler/cop_handler.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "fmt" + "strings" "time" "github.com/golang/protobuf/proto" @@ -85,7 +86,32 @@ type dagContext struct { startTS uint64 } -// handleCopDAGRequest handles coprocessor DAG request. +// ExecutorListsToTree converts a list of executors to a tree. +func ExecutorListsToTree(exec []*tipb.Executor) *tipb.Executor { + i := len(exec) - 1 + rootExec := exec[i] + for i--; 0 <= i; i-- { + switch exec[i+1].Tp { + case tipb.ExecType_TypeAggregation: + exec[i+1].Aggregation.Child = exec[i] + case tipb.ExecType_TypeProjection: + exec[i+1].Projection.Child = exec[i] + case tipb.ExecType_TypeTopN: + exec[i+1].TopN.Child = exec[i] + case tipb.ExecType_TypeLimit: + exec[i+1].Limit.Child = exec[i] + case tipb.ExecType_TypeSelection: + exec[i+1].Selection.Child = exec[i] + case tipb.ExecType_TypeStreamAgg: + exec[i+1].Aggregation.Child = exec[i] + default: + panic("unsupported dag executor type") + } + } + return rootExec +} + +// handleCopDAGRequest handles coprocessor DAG request using MPP executors. func handleCopDAGRequest(dbReader *dbreader.DBReader, lockStore *lockstore.MemStore, req *coprocessor.Request) (resp *coprocessor.Response) { startTime := time.Now() resp = &coprocessor.Response{} @@ -113,12 +139,88 @@ func handleCopDAGRequest(dbReader *dbreader.DBReader, lockStore *lockstore.MemSt resp.OtherError = err.Error() return resp } - closureExec, err := buildClosureExecutor(dagCtx, dagReq) + + exec, chunks, counts, ndvs, err := buildAndRunMPPExecutor(dagCtx, dagReq) + if err != nil { - return buildResp(nil, nil, nil, dagReq, err, dagCtx.sc.GetWarnings(), time.Since(startTime)) + errMsg := err.Error() + if strings.HasPrefix(errMsg, ErrExecutorNotSupportedMsg) { + resp.OtherError = err.Error() + return resp + } + return buildRespWithMPPExec(nil, nil, nil, exec, dagReq, err, dagCtx.sc.GetWarnings(), time.Since(startTime)) + } + return buildRespWithMPPExec(chunks, counts, ndvs, exec, dagReq, err, dagCtx.sc.GetWarnings(), time.Since(startTime)) +} + +func buildAndRunMPPExecutor(dagCtx *dagContext, dagReq *tipb.DAGRequest) (mppExec, []tipb.Chunk, []int64, []int64, error) { + rootExec := dagReq.RootExecutor + if rootExec == nil { + rootExec = ExecutorListsToTree(dagReq.Executors) + } + + var counts, ndvs []int64 + + if dagReq.GetCollectRangeCounts() { + counts = make([]int64, len(dagCtx.keyRanges)) + ndvs = make([]int64, len(dagCtx.keyRanges)) + } + builder := &mppExecBuilder{ + sc: dagCtx.sc, + dbReader: dagCtx.dbReader, + dagReq: dagReq, + dagCtx: dagCtx, + mppCtx: nil, + counts: counts, + ndvs: ndvs, + } + exec, err := builder.buildMPPExecutor(rootExec) + if err != nil { + return nil, nil, nil, nil, err + } + chunks, err := mppExecute(exec, dagCtx, dagReq) + return exec, chunks, counts, ndvs, err +} + +func mppExecute(exec mppExec, dagCtx *dagContext, dagReq *tipb.DAGRequest) (chunks []tipb.Chunk, err error) { + err = exec.open() + defer func() { + err := exec.stop() + if err != nil { + panic(err) + } + }() + if err != nil { + return + } + var buf []byte + var datums []types.Datum + var chk *chunk.Chunk + fields := exec.getFieldTypes() + for { + chk, err = exec.next() + if err != nil || chk == nil || chk.NumRows() == 0 { + return + } + numRows := chk.NumRows() + for i := 0; i < numRows; i++ { + datums = datums[:0] + if dagReq.OutputOffsets != nil { + for _, j := range dagReq.OutputOffsets { + datums = append(datums, chk.GetRow(i).GetDatum(int(j), fields[j])) + } + } else { + for j, ft := range fields { + datums = append(datums, chk.GetRow(i).GetDatum(j, ft)) + } + } + buf, err = codec.EncodeValue(dagCtx.sc, buf[:0], datums...) + if err != nil { + return nil, errors.Trace(err) + } + chunks = appendRow(chunks, buf, i) + } } - chunks, err := closureExec.execute() - return buildResp(chunks, closureExec, closureExec.ndvs, dagReq, err, dagCtx.sc.GetWarnings(), time.Since(startTime)) } func buildDAG(reader *dbreader.DBReader, lockStore *lockstore.MemStore, req *coprocessor.Request) (*dagContext, *tipb.DAGRequest, error) { @@ -287,12 +389,8 @@ func (e *ErrLocked) Error() string { return fmt.Sprintf("key is locked, key: %q, Type: %v, primary: %q, startTS: %v", e.Key, e.LockType, e.Primary, e.StartTS) } -func buildResp(chunks []tipb.Chunk, closureExecutor *closureExecutor, ndvs []int64, dagReq *tipb.DAGRequest, err error, warnings []stmtctx.SQLWarn, dur time.Duration) *coprocessor.Response { +func buildRespWithMPPExec(chunks []tipb.Chunk, counts, ndvs []int64, exec mppExec, dagReq *tipb.DAGRequest, err error, warnings []stmtctx.SQLWarn, dur time.Duration) *coprocessor.Response { resp := &coprocessor.Response{} - var counts []int64 - if closureExecutor != nil { - counts = closureExecutor.counts - } selResp := &tipb.SelectResponse{ Error: toPBError(err), Chunks: chunks, @@ -301,34 +399,15 @@ func buildResp(chunks []tipb.Chunk, closureExecutor *closureExecutor, ndvs []int } executors := dagReq.Executors if dagReq.CollectExecutionSummaries != nil && *dagReq.CollectExecutionSummaries { + // for simplicity, we assume all executors to be spending the same amount of time as the request + timeProcessed := uint64(dur / time.Nanosecond) execSummary := make([]*tipb.ExecutorExecutionSummary, len(executors)) - for i := range execSummary { - if closureExecutor == nil { - selResp.ExecutionSummaries = execSummary - continue - } - switch executors[i].Tp { - case tipb.ExecType_TypeTableScan: - execSummary[i] = closureExecutor.scanCtx.execDetail.buildSummary() - case tipb.ExecType_TypeIndexScan: - execSummary[i] = closureExecutor.idxScanCtx.execDetail.buildSummary() - case tipb.ExecType_TypeSelection: - execSummary[i] = closureExecutor.selectionCtx.execDetail.buildSummary() - case tipb.ExecType_TypeTopN: - execSummary[i] = closureExecutor.topNCtx.execDetail.buildSummary() - case tipb.ExecType_TypeAggregation, tipb.ExecType_TypeStreamAgg: - execSummary[i] = closureExecutor.aggCtx.execDetail.buildSummary() - case tipb.ExecType_TypeLimit: - costNs := uint64(0) - rows := uint64(closureExecutor.rowCount) - numIter := uint64(1) - execSummary[i] = &tipb.ExecutorExecutionSummary{ - TimeProcessedNs: &costNs, - NumProducedRows: &rows, - NumIterations: &numIter, - } - default: - execSummary[i] = &tipb.ExecutorExecutionSummary{} + e := exec + for i := len(executors) - 1; 0 <= i; i-- { + execSummary[i] = e.buildSummary() + execSummary[i].TimeProcessedNs = &timeProcessed + if i != 0 { + e = exec.child() } } selResp.ExecutionSummaries = execSummary diff --git a/store/mockstore/unistore/cophandler/cop_handler_test.go b/store/mockstore/unistore/cophandler/cop_handler_test.go index fd4e244344f6f..e29f25a7eeb2e 100644 --- a/store/mockstore/unistore/cophandler/cop_handler_test.go +++ b/store/mockstore/unistore/cophandler/cop_handler_test.go @@ -93,7 +93,7 @@ func makeATestMutaion(op kvrpcpb.Op, key []byte, value []byte) *kvrpcpb.Mutation } } -func prepareTestTableData(t *testing.T, keyNumber int, tableID int64) *data { +func prepareTestTableData(keyNumber int, tableID int64) (*data, error) { stmtCtx := new(stmtctx.StatementContext) colIds := []int64{1, 2, 3} colTypes := []*types.FieldType{ @@ -105,8 +105,9 @@ func prepareTestTableData(t *testing.T, keyNumber int, tableID int64) *data { colTypeMap := map[int64]*types.FieldType{} for i := 0; i < 3; i++ { colInfos[i] = &tipb.ColumnInfo{ - ColumnId: colIds[i], - Tp: int32(colTypes[i].Tp), + ColumnId: colIds[i], + Tp: int32(colTypes[i].Tp), + Collation: -mysql.DefaultCollationID, } colTypeMap[colIds[i]] = colTypes[i] } @@ -117,7 +118,9 @@ func prepareTestTableData(t *testing.T, keyNumber int, tableID int64) *data { datum := types.MakeDatums(i, "abc", 10.0) rows[int64(i)] = datum rowEncodedData, err := tablecodec.EncodeRow(stmtCtx, datum, colIds, nil, nil, encoder) - require.NoError(t, err) + if err != nil { + return nil, err + } rowKeyEncodedData := tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(i)) encodedTestKVDatas[i] = &encodedTestKVData{encodedRowKey: rowKeyEncodedData, encodedRowValue: rowEncodedData} } @@ -126,7 +129,7 @@ func prepareTestTableData(t *testing.T, keyNumber int, tableID int64) *data { encodedTestKVDatas: encodedTestKVDatas, rows: rows, colTypes: colTypeMap, - } + }, nil } func getTestPointRange(tableID int64, handle int64) kv.KeyRange { @@ -202,8 +205,7 @@ func newDagContext(store *testStore, keyRanges []kv.KeyRange, dagReq *tipb.DAGRe // build and execute the executors according to the dagRequest and dagContext, // return the result chunk data, rows count and err if occurs. -func buildExecutorsAndExecute(dagRequest *tipb.DAGRequest, - dagCtx *dagContext) ([]tipb.Chunk, int, error) { +func buildExecutorsAndExecute(dagCtx *dagContext, dagRequest *tipb.DAGRequest) ([]tipb.Chunk, int, error) { closureExec, err := buildClosureExecutor(dagCtx, dagRequest) if err != nil { return nil, 0, err @@ -220,9 +222,10 @@ func buildExecutorsAndExecute(dagRequest *tipb.DAGRequest, // dagBuilder is used to build dag request type dagBuilder struct { - startTs uint64 - executors []*tipb.Executor - outputOffsets []uint32 + startTs uint64 + executors []*tipb.Executor + outputOffsets []uint32 + collectRangeCounts bool } // return a default dagBuilder @@ -230,6 +233,11 @@ func newDagBuilder() *dagBuilder { return &dagBuilder{executors: make([]*tipb.Executor, 0)} } +func (dagBuilder *dagBuilder) setCollectRangeCounts(collectRangeCounts bool) *dagBuilder { + dagBuilder.collectRangeCounts = collectRangeCounts + return dagBuilder +} + func (dagBuilder *dagBuilder) setStartTs(startTs uint64) *dagBuilder { dagBuilder.startTs = startTs return dagBuilder @@ -272,8 +280,9 @@ func (dagBuilder *dagBuilder) addLimit(limit uint64) *dagBuilder { func (dagBuilder *dagBuilder) build() *tipb.DAGRequest { return &tipb.DAGRequest{ - Executors: dagBuilder.executors, - OutputOffsets: dagBuilder.outputOffsets, + Executors: dagBuilder.executors, + OutputOffsets: dagBuilder.outputOffsets, + CollectRangeCounts: &dagBuilder.collectRangeCounts, } } @@ -296,9 +305,14 @@ func TestPointGet(t *testing.T) { // here would build mvccStore and server, and prepare // three rows data, just like the test data of table_scan.rs. // then init the store with the generated data. - data := prepareTestTableData(t, keyNumber, tableID) - store, clean := newTestStore(t, "cop_handler_test_db", "cop_handler_test_log") - defer clean() + data, err := prepareTestTableData(keyNumber, tableID) + require.NoError(t, err) + store, clean, err := newTestStore("cop_handler_test_db", "cop_handler_test_log") + require.NoError(t, err) + defer func() { + err := clean() + require.NoError(t, err) + }() errs := initTestData(store, data.encodedTestKVDatas) require.Nil(t, errs) @@ -312,7 +326,7 @@ func TestPointGet(t *testing.T) { build() dagCtx := newDagContext(store, []kv.KeyRange{getTestPointRange(tableID, handle)}, dagRequest, dagRequestStartTs) - chunks, rowCount, err := buildExecutorsAndExecute(dagRequest, dagCtx) + chunks, rowCount, err := buildExecutorsAndExecute(dagCtx, dagRequest) require.Len(t, chunks, 0) require.NoError(t, err) require.Equal(t, 0, rowCount) @@ -326,7 +340,7 @@ func TestPointGet(t *testing.T) { build() dagCtx = newDagContext(store, []kv.KeyRange{getTestPointRange(tableID, handle)}, dagRequest, dagRequestStartTs) - chunks, rowCount, err = buildExecutorsAndExecute(dagRequest, dagCtx) + chunks, rowCount, err = buildExecutorsAndExecute(dagCtx, dagRequest) require.NoError(t, err) require.Equal(t, 1, rowCount) returnedRow, err := codec.Decode(chunks[0].RowsData, 2) @@ -345,9 +359,14 @@ func TestPointGet(t *testing.T) { } func TestClosureExecutor(t *testing.T) { - data := prepareTestTableData(t, keyNumber, tableID) - store, clean := newTestStore(t, "cop_handler_test_db", "cop_handler_test_log") - defer clean() + data, err := prepareTestTableData(keyNumber, tableID) + require.NoError(t, err) + store, clean, err := newTestStore("cop_handler_test_db", "cop_handler_test_log") + require.NoError(t, err) + defer func() { + err := clean() + require.NoError(t, err) + }() errs := initTestData(store, data.encodedTestKVDatas) require.Nil(t, errs) @@ -362,12 +381,61 @@ func TestClosureExecutor(t *testing.T) { dagCtx := newDagContext(store, []kv.KeyRange{getTestPointRange(tableID, 1)}, dagRequest, dagRequestStartTs) - _, rowCount, err := buildExecutorsAndExecute(dagRequest, dagCtx) + _, rowCount, err := buildExecutorsAndExecute(dagCtx, dagRequest) require.NoError(t, err) require.Equal(t, 0, rowCount) } -func buildEQIntExpr(colID, val int64) *tipb.Expr { +func TestMppExecutor(t *testing.T) { + data, err := prepareTestTableData(keyNumber, tableID) + require.NoError(t, err) + store, clean, err := newTestStore("cop_handler_test_db", "cop_handler_test_log") + require.NoError(t, err) + defer func() { + err := clean() + require.NoError(t, err) + }() + + errs := initTestData(store, data.encodedTestKVDatas) + require.Nil(t, errs) + + dagRequest := newDagBuilder(). + setStartTs(dagRequestStartTs). + addTableScan(data.colInfos, tableID). + addSelection(buildEQIntExpr(1, 1)). + addLimit(1). + setOutputOffsets([]uint32{0, 1}). + setCollectRangeCounts(true). + build() + + dagCtx := newDagContext(store, []kv.KeyRange{getTestPointRange(tableID, 1)}, + dagRequest, dagRequestStartTs) + _, _, rowCount, _, err := buildAndRunMPPExecutor(dagCtx, dagRequest) + require.Equal(t, rowCount[0], int64(1)) + require.NoError(t, err) +} + +func buildNEIntExpr(colIdx, val int64) *tipb.Expr { + return &tipb.Expr{ + Tp: tipb.ExprType_ScalarFunc, + Sig: tipb.ScalarFuncSig_NEInt, + FieldType: expression.ToPBFieldType(types.NewFieldType(mysql.TypeLonglong)), + Children: []*tipb.Expr{ + { + Tp: tipb.ExprType_ColumnRef, + Val: codec.EncodeInt(nil, colIdx), + FieldType: expression.ToPBFieldType(types.NewFieldType(mysql.TypeLonglong)), + }, + { + Tp: tipb.ExprType_Int64, + Val: codec.EncodeInt(nil, val), + FieldType: expression.ToPBFieldType(types.NewFieldType(mysql.TypeLonglong)), + }, + }, + } +} + +func buildEQIntExpr(colIdx, val int64) *tipb.Expr { return &tipb.Expr{ Tp: tipb.ExprType_ScalarFunc, Sig: tipb.ScalarFuncSig_EQInt, @@ -375,7 +443,7 @@ func buildEQIntExpr(colID, val int64) *tipb.Expr { Children: []*tipb.Expr{ { Tp: tipb.ExprType_ColumnRef, - Val: codec.EncodeInt(nil, colID), + Val: codec.EncodeInt(nil, colIdx), FieldType: expression.ToPBFieldType(types.NewFieldType(mysql.TypeLonglong)), }, { @@ -431,27 +499,40 @@ func (ts *testStore) commit(keys [][]byte, startTS, commitTS uint64) error { }) } -func newTestStore(t *testing.T, dbPrefix string, logPrefix string) (*testStore, func()) { +func newTestStore(dbPrefix string, logPrefix string) (*testStore, func() error, error) { dbPath, err := os.MkdirTemp("", dbPrefix) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } LogPath, err := os.MkdirTemp("", logPrefix) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } db, err := createTestDB(dbPath, LogPath) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } // Some raft store path problems could not be found using simple store in tests // writer := NewDBWriter(dbBundle, safePoint) kvPath := filepath.Join(dbPath, "kv") raftPath := filepath.Join(dbPath, "raft") snapPath := filepath.Join(dbPath, "snap") err = os.MkdirAll(kvPath, os.ModePerm) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } err = os.MkdirAll(raftPath, os.ModePerm) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } err = os.Mkdir(snapPath, os.ModePerm) - require.NoError(t, err) + if err != nil { + return nil, nil, err + } - clean := func() { - require.NoError(t, db.Close()) + clean := func() error { + fmt.Printf("db closed") + return db.Close() } return &testStore{ @@ -459,7 +540,7 @@ func newTestStore(t *testing.T, dbPrefix string, logPrefix string) (*testStore, locks: lockstore.NewMemStore(4096), dbPath: dbPath, logPath: LogPath, - }, clean + }, clean, nil } func createTestDB(dbPath, LogPath string) (*badger.DB, error) { @@ -470,3 +551,81 @@ func createTestDB(dbPath, LogPath string) (*badger.DB, error) { opts.ManagedTxns = true return badger.Open(opts) } + +func BenchmarkExecutors(b *testing.B) { + + prepare := func(rows, limit int) (dagReq *tipb.DAGRequest, dagCtx *dagContext, clean func() error) { + data, err := prepareTestTableData(rows, tableID) + if err != nil { + b.Fatal(err) + } + store, clean, err := newTestStore(fmt.Sprintf("cop_handler_bench_db_%d_%d", rows, limit), "cop_handler_test_log") + if err != nil { + b.Fatal(err) + } + errs := initTestData(store, data.encodedTestKVDatas) + if len(errs) > 0 { + b.Fatal(errs) + } + + dagReq = newDagBuilder(). + setStartTs(dagRequestStartTs). + addTableScan(data.colInfos, tableID). + addSelection(buildNEIntExpr(0, 1)). + addLimit(uint64(limit)). + setOutputOffsets([]uint32{0, 1}). + setCollectRangeCounts(true). + build() + + dagCtx = newDagContext( + store, + []kv.KeyRange{ + { + StartKey: tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(0)), + EndKey: tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(rows)), + }, + }, + dagReq, + 3000000, + ) + return dagReq, dagCtx, clean + } + + rows := []int{1, 10, 100, 1000, 10000, 100000} + limit := []int{1, 10, 100, 1000, 10000, 100000} + cleanFuncs := make([]func() error, 0, len(rows)*len(limit)) + + for _, row := range rows { + for _, lim := range limit { + if lim > row { + break + } + dagReq, dagCtx, clean := prepare(row, lim) + cleanFuncs = append(cleanFuncs, clean) + + // b.Run(fmt.Sprintf("(row=%d, limit=%d)", row, lim), func(b *testing.B) { + // for i := 0; i < b.N; i++ { + // _, _, err := buildExecutorsAndExecute(dagCtx, dagReq) + // if err != nil { + // b.Fatal(err) + // } + // } + // + // }) + b.Run(fmt.Sprintf("(row=%d, limit=%d)", row, lim), func(b *testing.B) { + for i := 0; i < b.N; i++ { + _, _, _, _, err := buildAndRunMPPExecutor(dagCtx, dagReq) + if err != nil { + b.Fatal(err) + } + } + }) + } + } + for _, clean := range cleanFuncs { + err := clean() + if err != nil { + b.Fatal(err) + } + } +} diff --git a/store/mockstore/unistore/cophandler/mpp.go b/store/mockstore/unistore/cophandler/mpp.go index fa5e4196cc3be..59a66d31f6f88 100644 --- a/store/mockstore/unistore/cophandler/mpp.go +++ b/store/mockstore/unistore/cophandler/mpp.go @@ -25,12 +25,15 @@ import ( "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore/unistore/client" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/dbreader" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tipb/go-tipb" "go.uber.org/atomic" ) @@ -42,24 +45,38 @@ const ( MPPErrEstablishConnMultiTimes ) +const ( + // ErrExecutorNotSupportedMsg is the message for executor not supported. + ErrExecutorNotSupportedMsg = "executor not supported: " +) + type mppExecBuilder struct { sc *stmtctx.StatementContext dbReader *dbreader.DBReader - req *coprocessor.Request mppCtx *MPPCtx dagReq *tipb.DAGRequest + dagCtx *dagContext + counts []int64 + ndvs []int64 } func (b *mppExecBuilder) buildMPPTableScan(pb *tipb.TableScan) (*tableScanExec, error) { - ranges, err := extractKVRanges(b.dbReader.StartKey, b.dbReader.EndKey, b.req.Ranges, false) + ranges, err := extractKVRanges(b.dbReader.StartKey, b.dbReader.EndKey, b.dagCtx.keyRanges, pb.Desc) if err != nil { return nil, errors.Trace(err) } ts := &tableScanExec{ baseMPPExec: baseMPPExec{sc: b.sc, mppCtx: b.mppCtx}, - startTS: b.req.StartTs, + startTS: b.dagCtx.startTS, kvRanges: ranges, dbReader: b.dbReader, + counts: b.counts, + ndvs: b.ndvs, + desc: pb.Desc, + } + if b.dagCtx != nil { + ts.lockStore = b.dagCtx.lockStore + ts.resolvedLocks = b.dagCtx.resolvedLocks } for _, col := range pb.Columns { ft := fieldTypeFromPBColumn(col) @@ -69,6 +86,112 @@ func (b *mppExecBuilder) buildMPPTableScan(pb *tipb.TableScan) (*tableScanExec, return ts, err } +func (b *mppExecBuilder) buildIdxScan(pb *tipb.IndexScan) (*indexScanExec, error) { + ranges, err := extractKVRanges(b.dbReader.StartKey, b.dbReader.EndKey, b.dagCtx.keyRanges, pb.Desc) + if err != nil { + return nil, errors.Trace(err) + } + numCols := len(pb.Columns) + numIdxCols := numCols + colInfos := make([]rowcodec.ColInfo, 0, numCols) + fieldTypes := make([]*types.FieldType, 0, numCols) + primaryColIds := pb.GetPrimaryColumnIds() + + lastCol := pb.Columns[numCols-1] + if lastCol.GetColumnId() == model.ExtraPidColID { + lastCol = pb.Columns[numCols-2] + numIdxCols-- + } + + hdlStatus := tablecodec.HandleDefault + if len(primaryColIds) == 0 { + if lastCol.GetPkHandle() { + if mysql.HasUnsignedFlag(uint(lastCol.GetFlag())) { + hdlStatus = tablecodec.HandleIsUnsigned + } + numIdxCols-- + } else if lastCol.ColumnId == model.ExtraHandleID { + numIdxCols-- + } + } else { + numIdxCols -= len(primaryColIds) + } + + for _, col := range pb.Columns { + ft := fieldTypeFromPBColumn(col) + fieldTypes = append(fieldTypes, ft) + colInfos = append(colInfos, rowcodec.ColInfo{ + ID: col.ColumnId, + Ft: ft, + IsPKHandle: col.GetPkHandle(), + }) + } + + var prevVals [][]byte + if b.dagReq.GetCollectRangeCounts() { + prevVals = make([][]byte, numIdxCols) + } + idxScan := &indexScanExec{ + baseMPPExec: baseMPPExec{sc: b.sc, fieldTypes: fieldTypes}, + startTS: b.dagCtx.startTS, + kvRanges: ranges, + dbReader: b.dbReader, + lockStore: b.dagCtx.lockStore, + resolvedLocks: b.dagCtx.resolvedLocks, + counts: b.counts, + ndvs: b.ndvs, + prevVals: prevVals, + colInfos: colInfos, + numIdxCols: numIdxCols, + hdlStatus: hdlStatus, + desc: pb.Desc, + } + return idxScan, nil +} + +func (b *mppExecBuilder) buildLimit(pb *tipb.Limit) (*limitExec, error) { + child, err := b.buildMPPExecutor(pb.Child) + if err != nil { + return nil, err + } + exec := &limitExec{ + baseMPPExec: baseMPPExec{sc: b.sc, mppCtx: b.mppCtx, fieldTypes: child.getFieldTypes(), children: []mppExec{child}}, + limit: pb.GetLimit(), + } + return exec, nil +} + +func (b *mppExecBuilder) buildTopN(pb *tipb.TopN) (*topNExec, error) { + child, err := b.buildMPPExecutor(pb.Child) + if err != nil { + return nil, err + } + pbConds := make([]*tipb.Expr, len(pb.OrderBy)) + for i, item := range pb.OrderBy { + pbConds[i] = item.Expr + } + heap := &topNHeap{ + totalCount: int(pb.Limit), + topNSorter: topNSorter{ + orderByItems: pb.OrderBy, + sc: b.sc, + }, + } + fieldTps := child.getFieldTypes() + var conds []expression.Expression + if conds, err = convertToExprs(b.sc, fieldTps, pbConds); err != nil { + return nil, errors.Trace(err) + } + exec := &topNExec{ + baseMPPExec: baseMPPExec{sc: b.sc, mppCtx: b.mppCtx, fieldTypes: fieldTps, children: []mppExec{child}}, + heap: heap, + conds: conds, + row: newTopNSortRow(len(conds)), + topn: pb.Limit, + } + return exec, nil +} + func (b *mppExecBuilder) buildMPPExchangeSender(pb *tipb.ExchangeSender) (*exchSenderExec, error) { child, err := b.buildMPPExecutor(pb.Child) if err != nil { @@ -309,15 +432,21 @@ func (b *mppExecBuilder) buildMPPExecutor(exec *tipb.Executor) (mppExec, error) case tipb.ExecType_TypeJoin: join := exec.Join return b.buildMPPJoin(join, join.Children) - case tipb.ExecType_TypeAggregation: + case tipb.ExecType_TypeAggregation, tipb.ExecType_TypeStreamAgg: agg := exec.Aggregation return b.buildMPPAgg(agg) case tipb.ExecType_TypeProjection: return b.buildMPPProj(exec.Projection) case tipb.ExecType_TypeSelection: return b.buildMPPSel(exec.Selection) + case tipb.ExecType_TypeIndexScan: + return b.buildIdxScan(exec.IdxScan) + case tipb.ExecType_TypeLimit: + return b.buildLimit(exec.Limit) + case tipb.ExecType_TypeTopN: + return b.buildTopN(exec.TopN) default: - return nil, errors.Errorf("Do not support executor %s", exec.Tp.String()) + return nil, errors.Errorf(ErrExecutorNotSupportedMsg + exec.Tp.String()) } } @@ -329,12 +458,17 @@ func HandleMPPDAGReq(dbReader *dbreader.DBReader, req *coprocessor.Request, mppC if err != nil { return &coprocessor.Response{OtherError: err.Error()} } + dagCtx := &dagContext{ + dbReader: dbReader, + startTS: req.StartTs, + keyRanges: req.Ranges, + } builder := mppExecBuilder{ dbReader: dbReader, - req: req, mppCtx: mppCtx, sc: flagsToStatementContext(dagReq.Flags), dagReq: dagReq, + dagCtx: dagCtx, } mppExec, err := builder.buildMPPExecutor(dagReq.RootExecutor) if err != nil { diff --git a/store/mockstore/unistore/cophandler/mpp_exec.go b/store/mockstore/unistore/cophandler/mpp_exec.go index 9e2e071513e04..9752371d50c02 100644 --- a/store/mockstore/unistore/cophandler/mpp_exec.go +++ b/store/mockstore/unistore/cophandler/mpp_exec.go @@ -15,8 +15,11 @@ package cophandler import ( + "bytes" + "encoding/binary" "io" "math" + "sort" "sync" "time" @@ -28,9 +31,11 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/store/mockstore/unistore/lockstore" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/dbreader" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" @@ -38,11 +43,19 @@ import ( "github.com/tikv/client-go/v2/tikvrpc" ) +var ( + // DefaultBatchSize is the default batch size for newly allocated chunk during execution. + DefaultBatchSize = 32 +) + // mpp executor that only servers for mpp execution type mppExec interface { open() error next() (*chunk.Chunk, error) + stop() error + child() mppExec getFieldTypes() []*types.FieldType + buildSummary() *tipb.ExecutorExecutionSummary } type baseMPPExec struct { @@ -52,24 +65,64 @@ type baseMPPExec struct { children []mppExec - fieldTypes []*types.FieldType + fieldTypes []*types.FieldType + execSummary execDetail +} + +func (b *baseMPPExec) child() mppExec { + return b.children[0] } func (b *baseMPPExec) getFieldTypes() []*types.FieldType { return b.fieldTypes } +func (b *baseMPPExec) buildSummary() *tipb.ExecutorExecutionSummary { + return b.execSummary.buildSummary() +} + +func (b *baseMPPExec) open() error { + panic("not implemented") +} + +func (b *baseMPPExec) next() (*chunk.Chunk, error) { + panic("not implemented") +} + +func (b *baseMPPExec) stop() error { + for _, child := range b.children { + err := child.stop() + if err != nil { + return errors.Trace(err) + } + } + return nil +} + +type scanResult struct { + chk *chunk.Chunk + err error +} + type tableScanExec struct { baseMPPExec - kvRanges []kv.KeyRange - startTS uint64 - dbReader *dbreader.DBReader + kvRanges []kv.KeyRange + startTS uint64 + dbReader *dbreader.DBReader + lockStore *lockstore.MemStore + resolvedLocks []uint64 + counts []int64 + ndvs []int64 + rowCnt int64 - chunks []*chunk.Chunk - chkIdx int + chk *chunk.Chunk + result chan scanResult + done chan struct{} + wg util.WaitGroupWrapper decoder *rowcodec.ChunkDecoder + desc bool } func (e *tableScanExec) SkipValue() bool { return false } @@ -79,33 +132,283 @@ func (e *tableScanExec) Process(key, value []byte) error { if err != nil { return errors.Trace(err) } - chk := chunk.NewChunkWithCapacity(e.fieldTypes, 0) - err = e.decoder.DecodeToChunk(value, handle, chk) - e.chunks = append(e.chunks, chk) + + err = e.decoder.DecodeToChunk(value, handle, e.chk) if err != nil { return errors.Trace(err) } + e.rowCnt++ + + if e.chk.IsFull() { + select { + case e.result <- scanResult{chk: e.chk, err: nil}: + e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) + case <-e.done: + return dbreader.ErrScanBreak + } + } + select { + case <-e.done: + return dbreader.ErrScanBreak + default: + } return nil } func (e *tableScanExec) open() error { + var err error + if e.lockStore != nil { + for _, ran := range e.kvRanges { + err = checkRangeLockForRange(e.lockStore, e.startTS, e.resolvedLocks, ran) + if err != nil { + return err + } + } + } + e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) + e.result = make(chan scanResult, 1) + e.done = make(chan struct{}) + e.wg.Run(func() { + // close the channel when done scanning, so that next() will got nil chunk + defer close(e.result) + for i, ran := range e.kvRanges { + oldCnt := e.rowCnt + if e.desc { + err = e.dbReader.ReverseScan(ran.StartKey, ran.EndKey, math.MaxInt64, e.startTS, e) + } else { + err = e.dbReader.Scan(ran.StartKey, ran.EndKey, math.MaxInt64, e.startTS, e) + } + if len(e.counts) != 0 { + e.counts[i] += e.rowCnt - oldCnt + e.ndvs[i] += e.rowCnt - oldCnt + } + if err != nil { + e.result <- scanResult{err: err} + return + } + } + + // handle the last chunk + if e.chk != nil && e.chk.NumRows() > 0 { + select { + case e.result <- scanResult{chk: e.chk, err: nil}: + return + case <-e.done: + } + return + } + }) + + return nil +} + +func (e *tableScanExec) next() (*chunk.Chunk, error) { + result := <-e.result + if result.chk == nil || result.err != nil { + return nil, result.err + } + e.execSummary.updateOnlyRows(result.chk.NumRows()) + return result.chk, nil +} + +func (e *tableScanExec) stop() error { + // just in case the channel is not initialized + if e.done != nil { + close(e.done) + } + e.wg.Wait() + return nil +} + +type indexScanExec struct { + baseMPPExec + + startTS uint64 + kvRanges []kv.KeyRange + desc bool + dbReader *dbreader.DBReader + lockStore *lockstore.MemStore + resolvedLocks []uint64 + counts []int64 + ndvs []int64 + prevVals [][]byte + rowCnt int64 + ndvCnt int64 + chk *chunk.Chunk + chkIdx int + chunks []*chunk.Chunk + + colInfos []rowcodec.ColInfo + numIdxCols int + hdlStatus tablecodec.HandleStatus +} + +func (e *indexScanExec) SkipValue() bool { return false } + +func (e *indexScanExec) isNewVals(values [][]byte) bool { + for i := 0; i < e.numIdxCols; i++ { + if !bytes.Equal(e.prevVals[i], values[i]) { + return true + } + } + return false +} + +func (e *indexScanExec) Process(key, value []byte) error { + values, err := tablecodec.DecodeIndexKV(key, value, e.numIdxCols, e.hdlStatus, e.colInfos) + if err != nil { + return err + } + e.rowCnt++ + if len(e.counts) > 0 && (len(e.prevVals[0]) == 0 || e.isNewVals(values)) { + e.ndvCnt++ + for i := 0; i < e.numIdxCols; i++ { + e.prevVals[i] = append(e.prevVals[i][:0], values[i]...) + } + } + decoder := codec.NewDecoder(e.chk, e.sc.TimeZone) + for i, value := range values { + if i < len(e.fieldTypes) { + _, err = decoder.DecodeOne(value, i, e.fieldTypes[i]) + if err != nil { + return errors.Trace(err) + } + } + } + if e.chk.IsFull() { + e.chunks = append(e.chunks, e.chk) + e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) + } + return nil +} + +func (e *indexScanExec) open() error { + var err error for _, ran := range e.kvRanges { - err := e.dbReader.Scan(ran.StartKey, ran.EndKey, math.MaxInt64, e.startTS, e) + err = checkRangeLockForRange(e.lockStore, e.startTS, e.resolvedLocks, ran) + if err != nil { + return err + } + } + e.chk = chunk.NewChunkWithCapacity(e.fieldTypes, DefaultBatchSize) + for i, rg := range e.kvRanges { + oldCnt := e.rowCnt + e.ndvCnt = 0 + if e.desc { + err = e.dbReader.ReverseScan(rg.StartKey, rg.EndKey, math.MaxInt64, e.startTS, e) + } else { + err = e.dbReader.Scan(rg.StartKey, rg.EndKey, math.MaxInt64, e.startTS, e) + } + if len(e.counts) != 0 { + e.counts[i] += e.rowCnt - oldCnt + e.ndvs[i] += e.ndvCnt + } if err != nil { return errors.Trace(err) } } + if e.chk.NumRows() != 0 { + e.chunks = append(e.chunks, e.chk) + } return nil } -func (e *tableScanExec) next() (*chunk.Chunk, error) { +func (e *indexScanExec) next() (*chunk.Chunk, error) { if e.chkIdx < len(e.chunks) { - e.chkIdx++ + e.chkIdx += 1 + e.execSummary.updateOnlyRows(e.chunks[e.chkIdx-1].NumRows()) return e.chunks[e.chkIdx-1], nil } return nil, nil } +type limitExec struct { + baseMPPExec + + limit uint64 +} + +func (e *limitExec) open() error { + return e.children[0].open() +} + +func (e *limitExec) next() (*chunk.Chunk, error) { + chk, err := e.children[0].next() + if err != nil || chk == nil || chk.NumRows() == 0 { + return chk, err + } + if uint64(chk.NumRows()) <= e.limit { + e.limit -= uint64(chk.NumRows()) + } else { + chk.TruncateTo(int(e.limit)) + e.limit = 0 + } + e.execSummary.updateOnlyRows(chk.NumRows()) + return chk, nil +} + +type topNExec struct { + baseMPPExec + + topn uint64 + idx uint64 + heap *topNHeap + conds []expression.Expression + row *sortRow + recv []*chunk.Chunk +} + +func (e *topNExec) open() error { + var chk *chunk.Chunk + var err error + err = e.children[0].open() + if err != nil { + return err + } + for { + chk, err = e.children[0].next() + if err != nil { + return err + } + if chk == nil || chk.NumRows() == 0 { + break + } + e.execSummary.updateOnlyRows(chk.NumRows()) + numRows := chk.NumRows() + for i := 0; i < numRows; i++ { + row := chk.GetRow(i) + for j, cond := range e.conds { + d, err := cond.Eval(row) + if err != nil { + return err + } + d.Copy(&e.row.key[j]) + } + if e.heap.tryToAddRow(e.row) { + e.row.data[0] = make([]byte, 4) + binary.LittleEndian.PutUint32(e.row.data[0], uint32(len(e.recv))) + e.row.data[1] = make([]byte, 4) + binary.LittleEndian.PutUint32(e.row.data[1], uint32(i)) + e.row = newTopNSortRow(len(e.conds)) + } + } + e.recv = append(e.recv, chk) + } + sort.Sort(&e.heap.topNSorter) + return nil +} + +func (e *topNExec) next() (*chunk.Chunk, error) { + chk := chunk.NewChunkWithCapacity(e.getFieldTypes(), DefaultBatchSize) + for ; !chk.IsFull() && e.idx < e.topn && e.idx < uint64(e.heap.heapSize); e.idx++ { + row := e.heap.rows[e.idx] + chkID := binary.LittleEndian.Uint32(row.data[0]) + rowID := binary.LittleEndian.Uint32(row.data[1]) + chk.AppendRow(e.recv[chkID].GetRow(int(rowID))) + } + return chk, nil +} + type exchSenderExec struct { baseMPPExec @@ -146,6 +449,10 @@ func (e *exchSenderExec) next() (*chunk.Chunk, error) { close(tunnel.ErrCh) close(tunnel.DataCh) } + err := e.stop() + if err != nil { + panic(err) + } }() for { chk, err := e.children[0].next() @@ -154,7 +461,7 @@ func (e *exchSenderExec) next() (*chunk.Chunk, error) { tunnel.ErrCh <- err } return nil, nil - } else if chk != nil { + } else if chk != nil && chk.NumRows() != 0 { if e.exchangeTp == tipb.ExchangeType_Hash { rows := chk.NumRows() targetChunks := make([]*chunk.Chunk, 0, len(e.tunnels)) @@ -366,7 +673,7 @@ func (e *joinExec) buildHashTable() error { if err != nil { return errors.Trace(err) } - if chk == nil { + if chk == nil || chk.NumRows() == 0 { return nil } rows := chk.NumRows() @@ -392,7 +699,7 @@ func (e *joinExec) fetchRows() (bool, error) { if err != nil { return false, errors.Trace(err) } - if chk == nil { + if chk == nil || chk.NumRows() == 0 { return true, nil } e.idx = 0 @@ -444,6 +751,18 @@ func (e *joinExec) open() error { return nil } +func (e *joinExec) stop() error { + err := e.buildChild.stop() + if err != nil { + return errors.Trace(err) + } + err = e.probeChild.stop() + if err != nil { + return errors.Trace(err) + } + return nil +} + func (e *joinExec) next() (*chunk.Chunk, error) { if !e.inited { e.inited = true @@ -491,7 +810,7 @@ func (e *aggExec) getGroupKey(row chunk.Row) (*chunk.MutRow, []byte, error) { if length == 0 { return nil, nil, nil } - key := make([]byte, 0, 32) + key := make([]byte, 0, DefaultBatchSize) gbyRow := chunk.MutRowFromTypes(e.groupByTypes) for i, item := range e.groupByExprs { v, err := item.Eval(row) @@ -526,7 +845,7 @@ func (e *aggExec) processAllRows() (*chunk.Chunk, error) { if err != nil { return nil, errors.Trace(err) } - if chk == nil { + if chk == nil || chk.NumRows() == 0 { break } rows := chk.NumRows() @@ -575,6 +894,7 @@ func (e *aggExec) processAllRows() (*chunk.Chunk, error) { } chk.AppendRow(newRow.ToRow()) } + e.execSummary.updateOnlyRows(chk.NumRows()) return chk, nil } @@ -597,42 +917,50 @@ func (e *selExec) open() error { } func (e *selExec) next() (*chunk.Chunk, error) { - chk, err := e.children[0].next() - if err != nil { - return nil, errors.Trace(err) - } - if chk == nil { - return nil, nil - } - for rows := chk.NumRows() - 1; rows >= 0; rows-- { - row := chk.GetRow(rows) - for _, cond := range e.conditions { - d, err := cond.Eval(row) - if err != nil { - return nil, errors.Trace(err) - } - - var passCheck bool - if d.IsNull() { - passCheck = false - } else { - isBool, err := d.ToBool(e.sc) + ret := chunk.NewChunkWithCapacity(e.getFieldTypes(), DefaultBatchSize) + for !ret.IsFull() { + chk, err := e.children[0].next() + if err != nil { + return nil, errors.Trace(err) + } + if chk == nil || chk.NumRows() == 0 { + break + } + numRows := chk.NumRows() + for rows := 0; rows < numRows; rows++ { + row := chk.GetRow(rows) + passCheck := true + for _, cond := range e.conditions { + d, err := cond.Eval(row) if err != nil { return nil, errors.Trace(err) } - isBool, err = expression.HandleOverflowOnSelection(e.sc, isBool, err) - if err != nil { - return nil, errors.Trace(err) + + if d.IsNull() { + passCheck = false + } else { + isBool, err := d.ToBool(e.sc) + if err != nil { + return nil, errors.Trace(err) + } + isBool, err = expression.HandleOverflowOnSelection(e.sc, isBool, err) + if err != nil { + return nil, errors.Trace(err) + } + passCheck = isBool != 0 + } + if !passCheck { + break } - passCheck = isBool != 0 } - if !passCheck { - chk.TruncateTo(rows) - break + if passCheck { + ret.AppendRow(row) + e.execSummary.updateOnlyRows(1) } } } - return chk, nil + + return ret, nil } type projExec struct { @@ -649,9 +977,10 @@ func (e *projExec) next() (*chunk.Chunk, error) { if err != nil { return nil, errors.Trace(err) } - if chk == nil { + if chk == nil || chk.NumRows() == 0 { return nil, nil } + e.baseMPPExec.execSummary.updateOnlyRows(chk.NumRows()) newChunk := chunk.NewChunkWithCapacity(e.fieldTypes, 10) for i := 0; i < chk.NumRows(); i++ { row := chk.GetRow(i) diff --git a/store/mockstore/unistore/pd.go b/store/mockstore/unistore/pd.go index 3ed8f53d3d9d2..a5129caf01a8d 100644 --- a/store/mockstore/unistore/pd.go +++ b/store/mockstore/unistore/pd.go @@ -32,15 +32,53 @@ type pdClient struct { serviceSafePoints map[string]uint64 gcSafePointMu sync.Mutex + globalConfig map[string]string } func newPDClient(pd *us.MockPD) *pdClient { return &pdClient{ MockPD: pd, serviceSafePoints: make(map[string]uint64), + globalConfig: make(map[string]string), } } +func (c *pdClient) LoadGlobalConfig(ctx context.Context, names []string) ([]pd.GlobalConfigItem, error) { + ret := make([]pd.GlobalConfigItem, len(names)) + for i, name := range names { + if r, ok := c.globalConfig["/global/config/"+name]; ok { + ret[i] = pd.GlobalConfigItem{Name: "/global/config/" + name, Value: r} + } else { + ret[i] = pd.GlobalConfigItem{Name: "/global/config/" + name, Error: errors.New("not found")} + } + } + return ret, nil +} + +func (c *pdClient) StoreGlobalConfig(ctx context.Context, items []pd.GlobalConfigItem) error { + for _, item := range items { + c.globalConfig["/global/config/"+item.Name] = item.Value + } + return nil +} + +func (c *pdClient) WatchGlobalConfig(ctx context.Context) (chan []pd.GlobalConfigItem, error) { + globalConfigWatcherCh := make(chan []pd.GlobalConfigItem, 16) + go func() { + defer func() { + if r := recover(); r != nil { + return + } + }() + for i := 0; i < 10; i++ { + for k, v := range c.globalConfig { + globalConfigWatcherCh <- []pd.GlobalConfigItem{{Name: k, Value: v}} + } + } + }() + return globalConfigWatcherCh, nil +} + func (c *pdClient) GetLocalTS(ctx context.Context, dcLocation string) (int64, int64, error) { return c.GetTS(ctx) } diff --git a/store/mockstore/unistore/pd_test.go b/store/mockstore/unistore/pd_test.go new file mode 100644 index 0000000000000..1fa645a3f2e86 --- /dev/null +++ b/store/mockstore/unistore/pd_test.go @@ -0,0 +1,96 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package unistore + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + pd "github.com/tikv/pd/client" +) + +type GlobalConfigTestSuite struct { + rpc *RPCClient + cluster *Cluster + client pd.Client +} + +func SetUpSuite() *GlobalConfigTestSuite { + s := &GlobalConfigTestSuite{} + s.rpc, s.client, s.cluster, _ = New("") + return s +} + +func TestLoad(t *testing.T) { + s := SetUpSuite() + + s.client.StoreGlobalConfig(context.Background(), []pd.GlobalConfigItem{{Name: "LoadOkGlobalConfig", Value: "ok"}}) + res, err := s.client.LoadGlobalConfig(context.Background(), []string{"LoadOkGlobalConfig", "LoadErrGlobalConfig"}) + require.Equal(t, err, nil) + for _, j := range res { + switch j.Name { + case "/global/config/LoadOkGlobalConfig": + require.Equal(t, j.Value, "ok") + + case "/global/config/LoadErrGlobalConfig": + require.Equal(t, j.Value, "") + require.EqualError(t, j.Error, "not found") + default: + require.Equal(t, true, false) + } + } + s.TearDownSuite() +} + +func TestStore(t *testing.T) { + s := SetUpSuite() + + res, err := s.client.LoadGlobalConfig(context.Background(), []string{"NewObject"}) + require.Equal(t, err, nil) + require.EqualError(t, res[0].Error, "not found") + + err = s.client.StoreGlobalConfig(context.Background(), []pd.GlobalConfigItem{{Name: "NewObject", Value: "ok"}}) + require.Equal(t, err, nil) + + res, err = s.client.LoadGlobalConfig(context.Background(), []string{"NewObject"}) + require.Equal(t, err, nil) + require.Equal(t, res[0].Error, nil) + + s.TearDownSuite() +} + +func TestWatch(t *testing.T) { + s := SetUpSuite() + err := s.client.StoreGlobalConfig(context.Background(), []pd.GlobalConfigItem{{Name: "NewObject", Value: "ok"}}) + require.Equal(t, err, nil) + + ch, err := s.client.WatchGlobalConfig(context.Background()) + require.Equal(t, err, nil) + + for i := 0; i < 10; i++ { + res := <-ch + require.NotEqual(t, res[0].Value, "") + } + close(ch) + + s.TearDownSuite() +} + +func (s *GlobalConfigTestSuite) TearDownSuite() { + s.client.Close() + s.rpc.Close() + s.cluster.Close() +} diff --git a/store/mockstore/unistore/tikv/dbreader/db_reader.go b/store/mockstore/unistore/tikv/dbreader/db_reader.go index 78ea3acb09e8c..66bdb6444ed01 100644 --- a/store/mockstore/unistore/tikv/dbreader/db_reader.go +++ b/store/mockstore/unistore/tikv/dbreader/db_reader.go @@ -198,13 +198,13 @@ func (r *DBReader) Scan(startKey, endKey []byte, limit int, startTS uint64, proc skipValue := proc.SkipValue() iter := r.GetIter() var cnt int + var err error for iter.Seek(startKey); iter.Valid(); iter.Next() { item := iter.Item() key := item.Key() if exceedEndKey(key, endKey) { break } - var err error if item.IsEmpty() { continue } diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index e44b5ca1fc0f1..d91c40406063a 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -53,20 +53,20 @@ func TestDeadlock(t *testing.T) { expireInterval := 100 * time.Millisecond urgentSize := uint64(1) detector := NewDetector(ttl, urgentSize, expireInterval) - err := detector.Detect(1, 2, 100, makeDiagCtx("k1", "tag1")) - require.Nil(t, err) + result := detector.Detect(1, 2, 100, makeDiagCtx("k1", "tag1")) + require.Nil(t, result) require.Equal(t, uint64(1), detector.totalSize) - err = detector.Detect(2, 3, 200, makeDiagCtx("k2", "tag2")) - require.Nil(t, err) + result = detector.Detect(2, 3, 200, makeDiagCtx("k2", "tag2")) + require.Nil(t, result) require.Equal(t, uint64(2), detector.totalSize) - err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) - require.NotNil(t, err) - require.Equal(t, "deadlock", err.Error()) - require.Equal(t, 3, len(err.WaitChain)) + result = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) + require.NotNil(t, result) + require.Equal(t, "deadlock", result.Error()) + require.Equal(t, 3, len(result.WaitChain)) // The order of entries in the wait chain is specific: each item is waiting for the next one. - checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") - checkWaitChainEntry(err.WaitChain[1], 2, 3, "k2", "tag2") - checkWaitChainEntry(err.WaitChain[2], 3, 1, "k3", "tag3") + checkWaitChainEntry(result.WaitChain[0], 1, 2, "k1", "tag1") + checkWaitChainEntry(result.WaitChain[1], 2, 3, "k2", "tag2") + checkWaitChainEntry(result.WaitChain[2], 3, 1, "k3", "tag3") require.Equal(t, uint64(2), detector.totalSize) detector.CleanUp(2) @@ -76,21 +76,21 @@ func TestDeadlock(t *testing.T) { // After cycle is broken, no deadlock now. diagCtx := diagnosticContext{} - err = detector.Detect(3, 1, 300, diagCtx) - require.Nil(t, err) + result = detector.Detect(3, 1, 300, diagCtx) + require.Nil(t, result) list3 := detector.waitForMap[3] require.Equal(t, 1, list3.txns.Len()) require.Equal(t, uint64(2), detector.totalSize) // Different keyHash grows the list. - err = detector.Detect(3, 1, 400, diagCtx) - require.Nil(t, err) + result = detector.Detect(3, 1, 400, diagCtx) + require.Nil(t, result) require.Equal(t, 2, list3.txns.Len()) require.Equal(t, uint64(3), detector.totalSize) // Same waitFor and key hash doesn't grow the list. - err = detector.Detect(3, 1, 400, diagCtx) - require.Nil(t, err) + result = detector.Detect(3, 1, 400, diagCtx) + require.Nil(t, result) require.Equal(t, 2, list3.txns.Len()) require.Equal(t, uint64(3), detector.totalSize) @@ -104,16 +104,16 @@ func TestDeadlock(t *testing.T) { // after 100ms, all entries expired, detect non exist edges time.Sleep(100 * time.Millisecond) - err = detector.Detect(100, 200, 100, diagCtx) - require.Nil(t, err) + result = detector.Detect(100, 200, 100, diagCtx) + require.Nil(t, result) require.Equal(t, uint64(1), detector.totalSize) require.Equal(t, 1, len(detector.waitForMap)) // expired entry should not report deadlock, detect will remove this entry // not dependent on expire check interval time.Sleep(60 * time.Millisecond) - err = detector.Detect(200, 100, 200, diagCtx) - require.Nil(t, err) + result = detector.Detect(200, 100, 200, diagCtx) + require.Nil(t, result) require.Equal(t, uint64(1), detector.totalSize) require.Equal(t, 1, len(detector.waitForMap)) } diff --git a/store/mockstore/unistore/tikv/errors.go b/store/mockstore/unistore/tikv/errors.go index 6043f2fd7a753..59635eb073a1b 100644 --- a/store/mockstore/unistore/tikv/errors.go +++ b/store/mockstore/unistore/tikv/errors.go @@ -15,6 +15,7 @@ package tikv import ( + "encoding/hex" "fmt" deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" @@ -132,3 +133,17 @@ type ErrTxnNotFound struct { func (e *ErrTxnNotFound) Error() string { return "txn not found" } + +// ErrAssertionFailed is returned if any assertion fails on a transaction request. +type ErrAssertionFailed struct { + StartTS uint64 + Key []byte + Assertion kvrpcpb.Assertion + ExistingStartTS uint64 + ExistingCommitTS uint64 +} + +func (e *ErrAssertionFailed) Error() string { + return fmt.Sprintf("AssertionFailed { StartTS: %v, Key: %v, Assertion: %v, ExistingStartTS: %v, ExistingCommitTS: %v }", + e.StartTS, hex.EncodeToString(e.Key), e.Assertion.String(), e.ExistingStartTS, e.ExistingCommitTS) +} diff --git a/store/mockstore/unistore/tikv/mock_region.go b/store/mockstore/unistore/tikv/mock_region.go index 9233447cf921d..416b28a1dffb2 100644 --- a/store/mockstore/unistore/tikv/mock_region.go +++ b/store/mockstore/unistore/tikv/mock_region.go @@ -356,11 +356,11 @@ func (rm *MockRegionManager) Split(regionID, newRegionID uint64, key []byte, pee // SplitRaw splits a Region at the key (not encoded) and creates new Region. func (rm *MockRegionManager) SplitRaw(regionID, newRegionID uint64, rawKey []byte, peerIDs []uint64, leaderPeerID uint64) *metapb.Region { - new, err := rm.split(regionID, newRegionID, rawKey, peerIDs) + r, err := rm.split(regionID, newRegionID, rawKey, peerIDs) if err != nil { panic(err) } - return proto.Clone(new).(*metapb.Region) + return proto.Clone(r).(*metapb.Region) } // SplitTable evenly splits the data in table into count regions. diff --git a/store/mockstore/unistore/tikv/mvcc.go b/store/mockstore/unistore/tikv/mvcc.go index a0d943a5647da..c7dccec121311 100644 --- a/store/mockstore/unistore/tikv/mvcc.go +++ b/store/mockstore/unistore/tikv/mvcc.go @@ -293,17 +293,23 @@ func (store *MVCCStore) PessimisticLock(reqCtx *requestCtx, req *kvrpcpb.Pessimi resp.Value = val resp.CommitTs = dbMeta.CommitTS() } - if req.ReturnValues { + if req.ReturnValues || req.CheckExistence { for _, item := range items { if item == nil { - resp.Values = append(resp.Values, nil) + if req.ReturnValues { + resp.Values = append(resp.Values, nil) + } + resp.NotFounds = append(resp.NotFounds, true) continue } val, err1 := item.ValueCopy(nil) if err1 != nil { return nil, err1 } - resp.Values = append(resp.Values, val) + if req.ReturnValues { + resp.Values = append(resp.Values, val) + } + resp.NotFounds = append(resp.NotFounds, len(val) == 0) } } return nil, err @@ -853,6 +859,36 @@ func (store *MVCCStore) buildPrewriteLock(reqCtx *requestCtx, m *kvrpcpb.Mutatio Value: m.Value, Secondaries: req.Secondaries, } + // Note that this is not fully consistent with TiKV. TiKV doesn't always get the value from Write CF. In + // AssertionLevel_Fast, TiKV skips checking assertion if Write CF is not read, in order not to harm the performance. + // However, unistore can always check it. It's better not to assume the store's behavior about assertion when the + // mode is set to AssertionLevel_Fast. + if req.AssertionLevel != kvrpcpb.AssertionLevel_Off { + if item == nil || item.IsEmpty() { + if m.Assertion == kvrpcpb.Assertion_Exist { + log.Error("ASSERTION FAIL!!! non-exist for must exist key", zap.Stringer("mutation", m)) + return nil, &ErrAssertionFailed{ + StartTS: req.StartVersion, + Key: m.Key, + Assertion: m.Assertion, + ExistingStartTS: 0, + ExistingCommitTS: 0, + } + } + } else { + if m.Assertion == kvrpcpb.Assertion_NotExist { + log.Error("ASSERTION FAIL!!! exist for must non-exist key", zap.Stringer("mutation", m)) + userMeta := mvcc.DBUserMeta(item.UserMeta()) + return nil, &ErrAssertionFailed{ + StartTS: req.StartVersion, + Key: m.Key, + Assertion: m.Assertion, + ExistingStartTS: userMeta.StartTS(), + ExistingCommitTS: userMeta.CommitTS(), + } + } + } + } var err error lock.Op = uint8(m.Op) if lock.Op == uint8(kvrpcpb.Op_Insert) { diff --git a/store/mockstore/unistore/tikv/mvcc_test.go b/store/mockstore/unistore/tikv/mvcc_test.go index fe389c8533e9f..f38772165fd39 100644 --- a/store/mockstore/unistore/tikv/mvcc_test.go +++ b/store/mockstore/unistore/tikv/mvcc_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/badger" "github.com/pingcap/badger/y" + "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/mockstore/unistore/config" @@ -90,10 +91,8 @@ func CreateTestDB(dbPath, LogPath string) (*badger.DB, error) { } func NewTestStore(dbPrefix string, logPrefix string, t *testing.T) (*TestStore, func()) { - dbPath, err := os.MkdirTemp("", dbPrefix) - require.NoError(t, err) - LogPath, err := os.MkdirTemp("", logPrefix) - require.NoError(t, err) + dbPath := t.TempDir() + LogPath := t.TempDir() safePoint := &SafePoint{} db, err := CreateTestDB(dbPath, LogPath) require.NoError(t, err) @@ -156,32 +155,54 @@ func PessimisticLock(pk []byte, key []byte, startTs uint64, lockTTL uint64, forU // PrewriteOptimistic raises optimistic prewrite requests on store func PrewriteOptimistic(pk []byte, key []byte, value []byte, startTs uint64, lockTTL uint64, minCommitTs uint64, useAsyncCommit bool, secondaries [][]byte, store *TestStore) error { + return PrewriteOptimisticWithAssertion(pk, key, value, startTs, lockTTL, minCommitTs, useAsyncCommit, secondaries, + kvrpcpb.Assertion_None, kvrpcpb.AssertionLevel_Off, store) +} + +// PrewriteOptimisticWithAssertion raises optimistic prewrite requests on store, with specified assertion config +func PrewriteOptimisticWithAssertion(pk []byte, key []byte, value []byte, startTs uint64, lockTTL uint64, + minCommitTs uint64, useAsyncCommit bool, secondaries [][]byte, assertion kvrpcpb.Assertion, + assertionLevel kvrpcpb.AssertionLevel, store *TestStore) error { op := kvrpcpb.Op_Put if value == nil { op = kvrpcpb.Op_Del } + mutation := newMutation(op, key, value) + mutation.Assertion = assertion prewriteReq := &kvrpcpb.PrewriteRequest{ - Mutations: []*kvrpcpb.Mutation{newMutation(op, key, value)}, + Mutations: []*kvrpcpb.Mutation{mutation}, PrimaryLock: pk, StartVersion: startTs, LockTtl: lockTTL, MinCommitTs: minCommitTs, UseAsyncCommit: useAsyncCommit, Secondaries: secondaries, + AssertionLevel: assertionLevel, } return store.MvccStore.prewriteOptimistic(store.newReqCtx(), prewriteReq.Mutations, prewriteReq) } -// PrewritePessimistic raises pessmistic prewrite requests +// PrewritePessimistic raises pessimistic prewrite requests func PrewritePessimistic(pk []byte, key []byte, value []byte, startTs uint64, lockTTL uint64, isPessimisticLock []bool, forUpdateTs uint64, store *TestStore) error { + return PrewritePessimisticWithAssertion(pk, key, value, startTs, lockTTL, isPessimisticLock, forUpdateTs, + kvrpcpb.Assertion_None, kvrpcpb.AssertionLevel_Off, store) +} + +// PrewritePessimisticWithAssertion raises pessimistic prewrite requests, with specified assertion config +func PrewritePessimisticWithAssertion(pk []byte, key []byte, value []byte, startTs uint64, lockTTL uint64, + isPessimisticLock []bool, forUpdateTs uint64, assertion kvrpcpb.Assertion, assertionLevel kvrpcpb.AssertionLevel, + store *TestStore) error { + mutation := newMutation(kvrpcpb.Op_Put, key, value) + mutation.Assertion = assertion prewriteReq := &kvrpcpb.PrewriteRequest{ - Mutations: []*kvrpcpb.Mutation{newMutation(kvrpcpb.Op_Put, key, value)}, + Mutations: []*kvrpcpb.Mutation{mutation}, PrimaryLock: pk, StartVersion: startTs, LockTtl: lockTTL, IsPessimisticLock: isPessimisticLock, ForUpdateTs: forUpdateTs, + AssertionLevel: assertionLevel, } return store.MvccStore.prewritePessimistic(store.newReqCtx(), prewriteReq.Mutations, prewriteReq) } @@ -486,8 +507,8 @@ func MustGetRollback(key []byte, ts uint64, store *TestStore) { } func TestBasicOptimistic(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() key1 := []byte("key1") val1 := []byte("val1") @@ -501,8 +522,8 @@ func TestBasicOptimistic(t *testing.T) { func TestPessimiticTxnTTL(t *testing.T) { var err error - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() // Pessimisitc lock key1 key1 := []byte("key1") @@ -529,8 +550,8 @@ func TestPessimiticTxnTTL(t *testing.T) { } func TestRollback(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() key := []byte("tkey") val := []byte("value") @@ -564,8 +585,8 @@ func TestRollback(t *testing.T) { func TestOverwritePessimisitcLock(t *testing.T) { var err error - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() key := []byte("key") startTs := uint64(1) @@ -592,8 +613,8 @@ func TestOverwritePessimisitcLock(t *testing.T) { func TestCheckTxnStatus(t *testing.T) { var err error - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() var resTTL, resCommitTs uint64 var action kvrpcpb.Action @@ -693,8 +714,8 @@ func TestCheckTxnStatus(t *testing.T) { } func TestCheckSecondaryLocksStatus(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() pk := []byte("pk") secondary := []byte("secondary") @@ -760,8 +781,8 @@ func TestCheckSecondaryLocksStatus(t *testing.T) { func TestMvccGet(t *testing.T) { var err error - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() lockTTL := uint64(100) pk := []byte("t1_r1") @@ -873,8 +894,8 @@ func TestMvccGet(t *testing.T) { } func TestPrimaryKeyOpLock(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() pk := func() []byte { return []byte("tpk") } val2 := []byte("val2") @@ -924,8 +945,8 @@ func TestPrimaryKeyOpLock(t *testing.T) { } func TestMvccTxnRead(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() // nothing at start k1 := []byte("tk1") @@ -995,8 +1016,8 @@ func TestMvccTxnRead(t *testing.T) { } func TestTxnPrewrite(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() // nothing at start k := []byte("tk") @@ -1030,8 +1051,8 @@ func TestTxnPrewrite(t *testing.T) { } func TestPrewriteInsert(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() // nothing at start k1 := []byte("tk1") @@ -1066,8 +1087,8 @@ func TestPrewriteInsert(t *testing.T) { } func TestRollbackKey(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("tk") v := []byte("v") @@ -1091,8 +1112,8 @@ func TestRollbackKey(t *testing.T) { } func TestCleanup(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("tk") v := []byte("v") @@ -1113,8 +1134,8 @@ func TestCleanup(t *testing.T) { } func TestCommit(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("tk") v := []byte("v") @@ -1164,8 +1185,8 @@ func TestCommit(t *testing.T) { } func TestMinCommitTs(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("tk") v := []byte("v") @@ -1185,8 +1206,8 @@ func TestMinCommitTs(t *testing.T) { } func TestPessimisticLock(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("tk") v := []byte("v") @@ -1350,8 +1371,8 @@ func TestPessimisticLock(t *testing.T) { } func TestResolveCommit(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() pk := []byte("tpk") v := []byte("v") @@ -1415,8 +1436,8 @@ func MustLoad(startTS, commitTS uint64, store *TestStore, pairs ...string) { } func TestBatchGet(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() MustLoad(100, 101, store, "ta:1", "tb:2", "tc:3") MustPrewritePut([]byte("ta"), []byte("ta"), []byte("0"), 103, store) keys := [][]byte{[]byte("ta"), []byte("tb"), []byte("tc")} @@ -1428,8 +1449,8 @@ func TestBatchGet(t *testing.T) { } func TestCommitPessimisticLock(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("ta") MustAcquirePessimisticLock(k, k, 10, 10, store) MustCommitErr(k, 20, 30, store) @@ -1438,8 +1459,8 @@ func TestCommitPessimisticLock(t *testing.T) { } func TestOpCheckNotExist(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("ta") v := []byte("v") @@ -1455,8 +1476,8 @@ func TestOpCheckNotExist(t *testing.T) { } func TestPessimisticLockForce(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k := []byte("ta") v := []byte("v") @@ -1472,8 +1493,8 @@ func TestPessimisticLockForce(t *testing.T) { } func TestScanSampleStep(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() for i := 0; i < 1000; i++ { k := genScanSampleStepKey(i) MustPrewritePut(k, k, k, 1, store) @@ -1505,8 +1526,8 @@ func genScanSampleStepKey(i int) []byte { } func TestAsyncCommitPrewrite(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() pk := []byte("tpk") pkVal := []byte("tpkVal") @@ -1534,8 +1555,8 @@ func TestAsyncCommitPrewrite(t *testing.T) { } func TestAccessCommittedLocks(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k0 := []byte("t0") v0 := []byte("v0") @@ -1630,8 +1651,8 @@ func TestAccessCommittedLocks(t *testing.T) { } func TestTiKVRCRead(t *testing.T) { - store, close := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) - defer close() + store, clean := NewTestStore("basic_optimistic_db", "basic_optimistic_log", t) + defer clean() k1 := []byte("t1") k2, v2 := []byte("t2"), []byte("v2") @@ -1680,3 +1701,102 @@ func TestTiKVRCRead(t *testing.T) { require.Equal(t, pair.Value, v) } } + +func TestAssertion(t *testing.T) { + store, clean := NewTestStore("TestAssertion", "TestAssertion", t) + defer clean() + + // Prepare + MustPrewriteOptimistic([]byte("k1"), []byte("k1"), []byte("v1"), 1, 100, 0, store) + MustPrewriteOptimistic([]byte("k1"), []byte("k2"), []byte("v2"), 1, 100, 0, store) + MustPrewriteOptimistic([]byte("k1"), []byte("k3"), []byte("v3"), 1, 100, 0, store) + MustCommit([]byte("k1"), 1, 2, store) + MustCommit([]byte("k2"), 1, 2, store) + MustCommit([]byte("k3"), 1, 2, store) + + checkAssertionFailedError := func(err error, disable bool, startTs uint64, key []byte, assertion kvrpcpb.Assertion, existingStartTs uint64, existingCommitTs uint64) { + t.Logf("Check error: %+q", err) + if disable { + require.Nil(t, err) + return + } + require.NotNil(t, err) + e, ok := errors.Cause(err).(*ErrAssertionFailed) + require.True(t, ok) + require.Equal(t, startTs, e.StartTS) + require.Equal(t, key, e.Key) + require.Equal(t, assertion, e.Assertion) + require.Equal(t, existingStartTs, e.ExistingStartTS) + require.Equal(t, existingCommitTs, e.ExistingCommitTS) + } + + for _, disable := range []bool{false, true} { + level := kvrpcpb.AssertionLevel_Strict + if disable { + level = kvrpcpb.AssertionLevel_Off + } + // Test with optimistic transaction + err := PrewriteOptimisticWithAssertion([]byte("k1"), []byte("k1"), []byte("v1"), 10, 100, 0, false, nil, + kvrpcpb.Assertion_NotExist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k1"), kvrpcpb.Assertion_NotExist, 1, 2) + err = PrewriteOptimisticWithAssertion([]byte("k11"), []byte("k11"), []byte("v11"), 10, 100, 0, false, nil, + kvrpcpb.Assertion_Exist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k11"), kvrpcpb.Assertion_Exist, 0, 0) + + // Test with pessimistic transaction + MustAcquirePessimisticLock([]byte("k2"), []byte("k2"), 10, 10, store) + err = PrewritePessimisticWithAssertion([]byte("k2"), []byte("k2"), []byte("v2"), 10, 100, []bool{true}, 10, + kvrpcpb.Assertion_NotExist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k2"), kvrpcpb.Assertion_NotExist, 1, 2) + MustAcquirePessimisticLock([]byte("k22"), []byte("k22"), 10, 10, store) + err = PrewritePessimisticWithAssertion([]byte("k22"), []byte("k22"), []byte("v22"), 10, 100, []bool{true}, 10, + kvrpcpb.Assertion_Exist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k22"), kvrpcpb.Assertion_Exist, 0, 0) + + // Test with pessimistic transaction (non-pessimistic-lock) + err = PrewritePessimisticWithAssertion([]byte("pk"), []byte("k3"), []byte("v3"), 10, 100, []bool{false}, 10, + kvrpcpb.Assertion_NotExist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k3"), kvrpcpb.Assertion_NotExist, 1, 2) + err = PrewritePessimisticWithAssertion([]byte("pk"), []byte("k33"), []byte("v33"), 10, 100, []bool{false}, 10, + kvrpcpb.Assertion_Exist, level, store) + checkAssertionFailedError(err, disable, 10, []byte("k33"), kvrpcpb.Assertion_Exist, 0, 0) + } + + for _, k := range [][]byte{ + []byte("k1"), + []byte("k11"), + []byte("k2"), + []byte("k22"), + []byte("k3"), + []byte("k33"), + } { + MustRollbackKey(k, 10, store) + } + + // Test assertion passes + // Test with optimistic transaction + err := PrewriteOptimisticWithAssertion([]byte("k1"), []byte("k1"), []byte("v1"), 20, 100, 0, false, nil, + kvrpcpb.Assertion_Exist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) + err = PrewriteOptimisticWithAssertion([]byte("k11"), []byte("k11"), []byte("v11"), 20, 100, 0, false, nil, + kvrpcpb.Assertion_NotExist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) + + // Test with pessimistic transaction + MustAcquirePessimisticLock([]byte("k2"), []byte("k2"), 20, 10, store) + err = PrewritePessimisticWithAssertion([]byte("k2"), []byte("k2"), []byte("v2"), 20, 100, []bool{true}, 10, + kvrpcpb.Assertion_Exist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) + MustAcquirePessimisticLock([]byte("k22"), []byte("k22"), 20, 10, store) + err = PrewritePessimisticWithAssertion([]byte("k22"), []byte("k22"), []byte("v22"), 20, 100, []bool{true}, 10, + kvrpcpb.Assertion_NotExist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) + + // Test with pessimistic transaction (non-pessimistic-lock) + err = PrewritePessimisticWithAssertion([]byte("pk"), []byte("k3"), []byte("v3"), 20, 100, []bool{false}, 10, + kvrpcpb.Assertion_Exist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) + err = PrewritePessimisticWithAssertion([]byte("pk"), []byte("k33"), []byte("v33"), 20, 100, []bool{false}, 10, + kvrpcpb.Assertion_NotExist, kvrpcpb.AssertionLevel_Strict, store) + require.Nil(t, err) +} diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index 5ffe08384fcf2..44610ba963c86 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -1035,6 +1035,16 @@ func convertToKeyError(err error) *kvrpcpb.KeyError { PrimaryKey: x.PrimaryKey, }, } + case *ErrAssertionFailed: + return &kvrpcpb.KeyError{ + AssertionFailed: &kvrpcpb.AssertionFailed{ + StartTs: x.StartTS, + Key: x.Key, + Assertion: x.Assertion, + ExistingStartTs: x.ExistingStartTS, + ExistingCommitTs: x.ExistingCommitTS, + }, + } default: return &kvrpcpb.KeyError{ Abort: err.Error(), diff --git a/store/pdtypes/api.go b/store/pdtypes/api.go new file mode 100644 index 0000000000000..de163ab1ad0ed --- /dev/null +++ b/store/pdtypes/api.go @@ -0,0 +1,116 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package pdtypes contains type defines under PD. +// +// Mainly copied from PD repo to avoid direct dependency. +package pdtypes + +import ( + "time" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/kvproto/pkg/pdpb" +) + +// StoresInfo records stores' info. +type StoresInfo struct { + Count int `json:"count"` + Stores []*StoreInfo `json:"stores"` +} + +// StoreInfo contains information about a store. +type StoreInfo struct { + Store *MetaStore `json:"store"` + Status *StoreStatus `json:"status"` +} + +// MetaStore contains meta information about a store. +type MetaStore struct { + *metapb.Store + StateName string `json:"state_name"` +} + +// StoreStatus contains status about a store. +type StoreStatus struct { + Capacity ByteSize `json:"capacity"` + Available ByteSize `json:"available"` + UsedSize ByteSize `json:"used_size"` + LeaderCount int `json:"leader_count"` + LeaderWeight float64 `json:"leader_weight"` + LeaderScore float64 `json:"leader_score"` + LeaderSize int64 `json:"leader_size"` + RegionCount int `json:"region_count"` + RegionWeight float64 `json:"region_weight"` + RegionScore float64 `json:"region_score"` + RegionSize int64 `json:"region_size"` + SlowScore uint64 `json:"slow_score"` + SendingSnapCount uint32 `json:"sending_snap_count,omitempty"` + ReceivingSnapCount uint32 `json:"receiving_snap_count,omitempty"` + IsBusy bool `json:"is_busy,omitempty"` + StartTS *time.Time `json:"start_ts,omitempty"` + LastHeartbeatTS *time.Time `json:"last_heartbeat_ts,omitempty"` + Uptime *Duration `json:"uptime,omitempty"` +} + +// RegionsInfo contains some regions with the detailed region info. +type RegionsInfo struct { + Count int `json:"count"` + Regions []RegionInfo `json:"regions"` +} + +// RegionInfo records detail region info for api usage. +type RegionInfo struct { + ID uint64 `json:"id"` + StartKey string `json:"start_key"` + EndKey string `json:"end_key"` + RegionEpoch *metapb.RegionEpoch `json:"epoch,omitempty"` + Peers []MetaPeer `json:"peers,omitempty"` + + Leader MetaPeer `json:"leader,omitempty"` + DownPeers []PDPeerStats `json:"down_peers,omitempty"` + PendingPeers []MetaPeer `json:"pending_peers,omitempty"` + WrittenBytes uint64 `json:"written_bytes"` + ReadBytes uint64 `json:"read_bytes"` + WrittenKeys uint64 `json:"written_keys"` + ReadKeys uint64 `json:"read_keys"` + ApproximateSize int64 `json:"approximate_size"` + ApproximateKeys int64 `json:"approximate_keys"` + + ReplicationStatus *ReplicationStatus `json:"replication_status,omitempty"` +} + +// MetaPeer is api compatible with *metapb.Peer. +type MetaPeer struct { + *metapb.Peer + // RoleName is `Role.String()`. + // Since Role is serialized as int by json by default, + // introducing it will make the output of pd-ctl easier to identify Role. + RoleName string `json:"role_name"` + // IsLearner is `Role == "Learner"`. + // Since IsLearner was changed to Role in kvproto in 5.0, this field was introduced to ensure api compatibility. + IsLearner bool `json:"is_learner,omitempty"` +} + +// PDPeerStats is api compatible with *pdpb.PeerStats. +type PDPeerStats struct { + *pdpb.PeerStats + Peer MetaPeer `json:"peer"` +} + +// ReplicationStatus represents the replication mode status of the region. +type ReplicationStatus struct { + State string `json:"state"` + StateID uint64 `json:"state_id"` +} diff --git a/store/pdtypes/config.go b/store/pdtypes/config.go new file mode 100644 index 0000000000000..42cdb0e7acda7 --- /dev/null +++ b/store/pdtypes/config.go @@ -0,0 +1,45 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pdtypes + +// ReplicationConfig is the replication configuration. +type ReplicationConfig struct { + // MaxReplicas is the number of replicas for each region. + MaxReplicas uint64 `toml:"max-replicas" json:"max-replicas"` + + // The label keys specified the location of a store. + // The placement priorities is implied by the order of label keys. + // For example, ["zone", "rack"] means that we should place replicas to + // different zones first, then to different racks if we don't have enough zones. + LocationLabels StringSlice `toml:"location-labels" json:"location-labels"` + // StrictlyMatchLabel strictly checks if the label of TiKV is matched with LocationLabels. + StrictlyMatchLabel bool `toml:"strictly-match-label" json:"strictly-match-label,string"` + + // When PlacementRules feature is enabled. MaxReplicas, LocationLabels and IsolationLabels are not used any more. + EnablePlacementRules bool `toml:"enable-placement-rules" json:"enable-placement-rules,string"` + + // EnablePlacementRuleCache controls whether use cache during rule checker + EnablePlacementRulesCache bool `toml:"enable-placement-rules-cache" json:"enable-placement-rules-cache,string"` + + // IsolationLevel is used to isolate replicas explicitly and forcibly if it's not empty. + // Its value must be empty or one of LocationLabels. + // Example: + // location-labels = ["zone", "rack", "host"] + // isolation-level = "zone" + // With configuration like above, PD ensure that all replicas be placed in different zones. + // Even if a zone is down, PD will not try to make up replicas in other zone + // because other zones already have replicas on it. + IsolationLevel string `toml:"isolation-level" json:"isolation-level"` +} diff --git a/store/pdtypes/placement.go b/store/pdtypes/placement.go new file mode 100644 index 0000000000000..d491ea4f2bdb3 --- /dev/null +++ b/store/pdtypes/placement.go @@ -0,0 +1,81 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pdtypes + +// Rule is the placement rule that can be checked against a region. When +// applying rules (apply means schedule regions to match selected rules), the +// apply order is defined by the tuple [GroupIndex, GroupID, Index, ID]. +type Rule struct { + GroupID string `json:"group_id"` // mark the source that add the rule + ID string `json:"id"` // unique ID within a group + Index int `json:"index,omitempty"` // rule apply order in a group, rule with less ID is applied first when indexes are equal + Override bool `json:"override,omitempty"` // when it is true, all rules with less indexes are disabled + StartKey []byte `json:"-"` // range start key + StartKeyHex string `json:"start_key"` // hex format start key, for marshal/unmarshal + EndKey []byte `json:"-"` // range end key + EndKeyHex string `json:"end_key"` // hex format end key, for marshal/unmarshal + Role PeerRoleType `json:"role"` // expected role of the peers + Count int `json:"count"` // expected count of the peers + LabelConstraints []LabelConstraint `json:"label_constraints,omitempty"` // used to select stores to place peers + LocationLabels []string `json:"location_labels,omitempty"` // used to make peers isolated physically + IsolationLevel string `json:"isolation_level,omitempty"` // used to isolate replicas explicitly and forcibly + Version uint64 `json:"version,omitempty"` // only set at runtime, add 1 each time rules updated, begin from 0. + CreateTimestamp uint64 `json:"create_timestamp,omitempty"` // only set at runtime, recorded rule create timestamp +} + +// PeerRoleType is the expected peer type of the placement rule. +type PeerRoleType string + +const ( + // Voter can either match a leader peer or follower peer + Voter PeerRoleType = "voter" + // Leader matches a leader. + Leader PeerRoleType = "leader" + // Follower matches a follower. + Follower PeerRoleType = "follower" + // Learner matches a learner. + Learner PeerRoleType = "learner" +) + +// LabelConstraint is used to filter store when trying to place peer of a region. +type LabelConstraint struct { + Key string `json:"key,omitempty"` + Op LabelConstraintOp `json:"op,omitempty"` + Values []string `json:"values,omitempty"` +} + +// RuleGroup defines properties of a rule group. +type RuleGroup struct { + ID string `json:"id,omitempty"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` +} + +// LabelConstraintOp defines how a LabelConstraint matches a store. It can be one of +// 'in', 'notIn', 'exists', or 'notExists'. +type LabelConstraintOp string + +const ( + // In restricts the store label value should in the value list. + // If label does not exist, `in` is always false. + In LabelConstraintOp = "in" + // NotIn restricts the store label value should not in the value list. + // If label does not exist, `notIn` is always true. + NotIn LabelConstraintOp = "notIn" + // Exists restricts the store should have the label. + Exists LabelConstraintOp = "exists" + // NotExists restricts the store should not have the label. + NotExists LabelConstraintOp = "notExists" +) diff --git a/store/pdtypes/region_tree.go b/store/pdtypes/region_tree.go new file mode 100644 index 0000000000000..efff8599030b1 --- /dev/null +++ b/store/pdtypes/region_tree.go @@ -0,0 +1,82 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pdtypes + +import ( + "bytes" + "sort" + + "github.com/pingcap/kvproto/pkg/metapb" +) + +// Region is a mock of PD's core.RegionInfo. For testing purpose. +type Region struct { + Meta *metapb.Region + Leader *metapb.Peer +} + +// NewRegionInfo returns a new RegionInfo. +func NewRegionInfo(meta *metapb.Region, leader *metapb.Peer) *Region { + return &Region{Meta: meta, Leader: leader} +} + +// RegionTree is a mock of PD's region tree. For testing purpose. +type RegionTree struct { + Regions []*Region +} + +// SetRegion puts a region to region tree. +func (t *RegionTree) SetRegion(region *Region) { + rs := t.Regions[:0] + for _, r := range t.Regions { + if !overlap(r, region) { + rs = append(rs, r) + } + } + rs = append(rs, region) + t.Regions = rs +} + +// ScanRange scans regions intersecting [start key, end key), returns at most +// `limit` regions. limit <= 0 means no limit. +func (t *RegionTree) ScanRange(startKey, endKey []byte, limit int) []*Region { + sort.Slice(t.Regions, func(i, j int) bool { + return bytes.Compare(t.Regions[i].Meta.StartKey, t.Regions[j].Meta.StartKey) < 0 + }) + pivot := NewRegionInfo(&metapb.Region{StartKey: startKey, EndKey: endKey}, nil) + var res []*Region + for _, r := range t.Regions { + if overlap(r, pivot) && (limit == 0 || len(res) < limit) { + res = append(res, r) + } + } + return res +} + +func overlap(a, b *Region) bool { + // |----a----| + // |----b----| + if len(b.Meta.EndKey) > 0 && bytes.Compare(b.Meta.EndKey, a.Meta.StartKey) <= 0 { + return false + } + + // |----a----| + // |----b----| + if len(a.Meta.EndKey) > 0 && bytes.Compare(a.Meta.EndKey, b.Meta.StartKey) <= 0 { + return false + } + + return true +} diff --git a/store/pdtypes/statistics.go b/store/pdtypes/statistics.go new file mode 100644 index 0000000000000..83309da0c4a0d --- /dev/null +++ b/store/pdtypes/statistics.go @@ -0,0 +1,29 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pdtypes + +// RegionStats records a list of regions' statistics and distribution status. +type RegionStats struct { + Count int `json:"count"` + EmptyCount int `json:"empty_count"` + StorageSize int64 `json:"storage_size"` + StorageKeys int64 `json:"storage_keys"` + StoreLeaderCount map[uint64]int `json:"store_leader_count"` + StorePeerCount map[uint64]int `json:"store_peer_count"` + StoreLeaderSize map[uint64]int64 `json:"store_leader_size"` + StoreLeaderKeys map[uint64]int64 `json:"store_leader_keys"` + StorePeerSize map[uint64]int64 `json:"store_peer_size"` + StorePeerKeys map[uint64]int64 `json:"store_peer_keys"` +} diff --git a/store/pdtypes/typeutil.go b/store/pdtypes/typeutil.go new file mode 100644 index 0000000000000..5d026f40e9583 --- /dev/null +++ b/store/pdtypes/typeutil.go @@ -0,0 +1,120 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pdtypes + +import ( + "fmt" + "strconv" + "strings" + "time" + + "github.com/docker/go-units" + "github.com/pingcap/errors" +) + +// ByteSize is a retype uint64 for TOML and JSON. +type ByteSize uint64 + +// MarshalJSON returns the size as a JSON string. +func (b ByteSize) MarshalJSON() ([]byte, error) { + return []byte(`"` + units.BytesSize(float64(b)) + `"`), nil +} + +// UnmarshalJSON parses a JSON string into the byte size. +func (b *ByteSize) UnmarshalJSON(text []byte) error { + s, err := strconv.Unquote(string(text)) + if err != nil { + return errors.WithStack(err) + } + v, err := units.RAMInBytes(s) + if err != nil { + return errors.WithStack(err) + } + *b = ByteSize(v) + return nil +} + +// UnmarshalText parses a Toml string into the byte size. +func (b *ByteSize) UnmarshalText(text []byte) error { + v, err := units.RAMInBytes(string(text)) + if err != nil { + return errors.WithStack(err) + } + *b = ByteSize(v) + return nil +} + +// Duration is a wrapper of time.Duration for TOML and JSON. +type Duration struct { + time.Duration +} + +// NewDuration creates a Duration from time.Duration. +func NewDuration(duration time.Duration) Duration { + return Duration{Duration: duration} +} + +// MarshalJSON returns the duration as a JSON string. +func (d *Duration) MarshalJSON() ([]byte, error) { + return []byte(fmt.Sprintf(`"%s"`, d.String())), nil +} + +// UnmarshalJSON parses a JSON string into the duration. +func (d *Duration) UnmarshalJSON(text []byte) error { + s, err := strconv.Unquote(string(text)) + if err != nil { + return errors.WithStack(err) + } + duration, err := time.ParseDuration(s) + if err != nil { + return errors.WithStack(err) + } + d.Duration = duration + return nil +} + +// UnmarshalText parses a TOML string into the duration. +func (d *Duration) UnmarshalText(text []byte) error { + var err error + d.Duration, err = time.ParseDuration(string(text)) + return errors.WithStack(err) +} + +// MarshalText returns the duration as a JSON string. +func (d Duration) MarshalText() ([]byte, error) { + return []byte(d.String()), nil +} + +// StringSlice is more friendly to json encode/decode +type StringSlice []string + +// MarshalJSON returns the size as a JSON string. +func (s StringSlice) MarshalJSON() ([]byte, error) { + return []byte(strconv.Quote(strings.Join(s, ","))), nil +} + +// UnmarshalJSON parses a JSON string into the byte size. +func (s *StringSlice) UnmarshalJSON(text []byte) error { + data, err := strconv.Unquote(string(text)) + if err != nil { + return errors.WithStack(err) + } + if len(data) == 0 { + *s = []string{} + return nil + } + *s = strings.Split(data, ",") + return nil +} diff --git a/store/store_test.go b/store/store_test.go index 2bad71202729b..698f3d4714ce3 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -23,10 +23,10 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" "github.com/stretchr/testify/require" + kv2 "github.com/tikv/client-go/v2/kv" ) const ( @@ -609,7 +609,7 @@ func TestDBClose(t *testing.T) { ver, err := store.CurrentVersion(kv.GlobalTxnScope) require.NoError(t, err) - require.Equal(t, 1, kv.MaxVersion.Cmp(ver), Equals) + require.Equal(t, 1, kv.MaxVersion.Cmp(ver)) snap := store.GetSnapshot(kv.MaxVersion) @@ -765,3 +765,109 @@ func TestRegister(t *testing.T) { err = Register("retry", &brokenStore{}) require.Error(t, err) } + +func TestSetAssertion(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + defer func() { + require.NoError(t, store.Close()) + }() + + txn, err := store.Begin() + require.NoError(t, err) + + mustHaveAssertion := func(key []byte, assertion kv.FlagsOp) { + f, err1 := txn.GetMemBuffer().GetFlags(key) + require.NoError(t, err1) + if assertion == kv.SetAssertExist { + require.True(t, f.HasAssertExists()) + require.False(t, f.HasAssertUnknown()) + } else if assertion == kv.SetAssertNotExist { + require.True(t, f.HasAssertNotExists()) + require.False(t, f.HasAssertUnknown()) + } else if assertion == kv.SetAssertUnknown { + require.True(t, f.HasAssertUnknown()) + } else if assertion == kv.SetAssertNone { + require.False(t, f.HasAssertionFlags()) + } else { + require.FailNow(t, "unreachable") + } + } + + testUnchangeable := func(key []byte, expectAssertion kv.FlagsOp) { + err = txn.SetAssertion(key, kv.SetAssertExist) + require.NoError(t, err) + mustHaveAssertion(key, expectAssertion) + err = txn.SetAssertion(key, kv.SetAssertNotExist) + require.NoError(t, err) + mustHaveAssertion(key, expectAssertion) + err = txn.SetAssertion(key, kv.SetAssertUnknown) + require.NoError(t, err) + mustHaveAssertion(key, expectAssertion) + err = txn.SetAssertion(key, kv.SetAssertNone) + require.NoError(t, err) + mustHaveAssertion(key, expectAssertion) + } + + k1 := []byte("k1") + err = txn.SetAssertion(k1, kv.SetAssertExist) + require.NoError(t, err) + mustHaveAssertion(k1, kv.SetAssertExist) + testUnchangeable(k1, kv.SetAssertExist) + + k2 := []byte("k2") + err = txn.SetAssertion(k2, kv.SetAssertNotExist) + require.NoError(t, err) + mustHaveAssertion(k2, kv.SetAssertNotExist) + testUnchangeable(k2, kv.SetAssertNotExist) + + k3 := []byte("k3") + err = txn.SetAssertion(k3, kv.SetAssertUnknown) + require.NoError(t, err) + mustHaveAssertion(k3, kv.SetAssertUnknown) + testUnchangeable(k3, kv.SetAssertUnknown) + + k4 := []byte("k4") + err = txn.SetAssertion(k4, kv.SetAssertNone) + require.NoError(t, err) + mustHaveAssertion(k4, kv.SetAssertNone) + err = txn.SetAssertion(k4, kv.SetAssertExist) + require.NoError(t, err) + mustHaveAssertion(k4, kv.SetAssertExist) + testUnchangeable(k4, kv.SetAssertExist) + + k5 := []byte("k5") + err = txn.Set(k5, []byte("v5")) + require.NoError(t, err) + mustHaveAssertion(k5, kv.SetAssertNone) + err = txn.SetAssertion(k5, kv.SetAssertNotExist) + require.NoError(t, err) + mustHaveAssertion(k5, kv.SetAssertNotExist) + testUnchangeable(k5, kv.SetAssertNotExist) + + k6 := []byte("k6") + err = txn.SetAssertion(k6, kv.SetAssertNotExist) + require.NoError(t, err) + err = txn.GetMemBuffer().SetWithFlags(k6, []byte("v6"), kv.SetPresumeKeyNotExists) + require.NoError(t, err) + mustHaveAssertion(k6, kv.SetAssertNotExist) + testUnchangeable(k6, kv.SetAssertNotExist) + flags, err := txn.GetMemBuffer().GetFlags(k6) + require.NoError(t, err) + require.True(t, flags.HasPresumeKeyNotExists()) + err = txn.GetMemBuffer().DeleteWithFlags(k6, kv.SetNeedLocked) + mustHaveAssertion(k6, kv.SetAssertNotExist) + testUnchangeable(k6, kv.SetAssertNotExist) + flags, err = txn.GetMemBuffer().GetFlags(k6) + require.NoError(t, err) + require.True(t, flags.HasPresumeKeyNotExists()) + require.True(t, flags.HasNeedLocked()) + + k7 := []byte("k7") + lockCtx := kv2.NewLockCtx(txn.StartTS(), 2000, time.Now()) + err = txn.LockKeys(context.Background(), lockCtx, k7) + require.NoError(t, err) + mustHaveAssertion(k7, kv.SetAssertNone) + + require.NoError(t, txn.Rollback()) +} diff --git a/structure/hash.go b/structure/hash.go index b0e7b15ca526f..7a40172c98ff7 100644 --- a/structure/hash.go +++ b/structure/hash.go @@ -166,6 +166,17 @@ func (t *TxStructure) HGetAll(key []byte) ([]HashPair, error) { return res, errors.Trace(err) } +// HGetLen gets the length of hash. +func (t *TxStructure) HGetLen(key []byte) (uint64, error) { + hashLen := 0 + err := t.iterateHash(key, func(field []byte, value []byte) error { + hashLen++ + return nil + }) + + return uint64(hashLen), errors.Trace(err) +} + // HGetLastN gets latest N fields and values in hash. func (t *TxStructure) HGetLastN(key []byte, num int) ([]HashPair, error) { res := make([]HashPair, 0, num) diff --git a/structure/main_test.go b/structure/main_test.go index 3874644b636a8..4f72134659832 100644 --- a/structure/main_test.go +++ b/structure/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/table/column.go b/table/column.go index 35ad165415c89..67c257a8476c4 100644 --- a/table/column.go +++ b/table/column.go @@ -561,7 +561,7 @@ func getColDefaultValue(ctx sessionctx.Context, col *model.ColumnInfo, defaultVa defer func() { sc.TimeZone = originalTZ }() } } - value, err := expression.GetTimeValue(ctx, defaultVal, col.Tp, int8(col.Decimal)) + value, err := expression.GetTimeValue(ctx, defaultVal, col.Tp, col.Decimal) if err != nil { return types.Datum{}, errGetDefaultFailed.GenWithStackByArgs(col.Name) } diff --git a/table/index.go b/table/index.go index d48ccf89501d1..62892c135b7c3 100644 --- a/table/index.go +++ b/table/index.go @@ -32,8 +32,9 @@ type IndexIterator interface { // CreateIdxOpt contains the options will be used when creating an index. type CreateIdxOpt struct { - Ctx context.Context - Untouched bool // If true, the index key/value is no need to commit. + Ctx context.Context + Untouched bool // If true, the index key/value is no need to commit. + IgnoreAssertion bool } // CreateIdxOptFunc is defined for the Create() method of Index interface. @@ -46,6 +47,11 @@ var IndexIsUntouched CreateIdxOptFunc = func(opt *CreateIdxOpt) { opt.Untouched = true } +// WithIgnoreAssertion uses to indicate the process can ignore assertion. +var WithIgnoreAssertion = func(opt *CreateIdxOpt) { + opt.IgnoreAssertion = true +} + // WithCtx returns a CreateIdxFunc. // This option is used to pass context.Context. func WithCtx(ctx context.Context) CreateIdxOptFunc { @@ -62,16 +68,10 @@ type Index interface { Create(ctx sessionctx.Context, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error - // Drop supports drop table, drop index statements. - Drop(txn kv.Transaction) error // Exist supports check index exists or not. Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) - // Seek supports where clause. - Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter IndexIterator, hit bool, err error) - // SeekFirst supports aggregate min and ascend order by. - SeekFirst(r kv.Retriever) (iter IndexIterator, err error) // FetchValues fetched index column values in a row. // Param columns is a reused buffer, if it is not nil, FetchValues will fill the index values in it, // and return the buffer, if it is nil, FetchValues will allocate the buffer instead. diff --git a/table/table.go b/table/table.go index d3b00748b2c95..cc5056986bf5c 100644 --- a/table/table.go +++ b/table/table.go @@ -241,6 +241,7 @@ type PartitionedTable interface { GetPartition(physicalID int64) PhysicalTable GetPartitionByRow(sessionctx.Context, []types.Datum) (PhysicalTable, error) GetAllPartitionIDs() []int64 + GetPartitionColumnNames() []model.CIStr } // TableFromMeta builds a table.Table from *model.TableInfo. diff --git a/table/tables/cache_test.go b/table/tables/cache_test.go index 8b9d716076c79..27ede985a76e7 100644 --- a/table/tables/cache_test.go +++ b/table/tables/cache_test.go @@ -20,14 +20,20 @@ import ( "time" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/stmtsummary" + dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" ) +func lastReadFromCache(tk *testkit.TestKit) bool { + return tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache +} + func TestCacheTableBasicScan(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() @@ -40,114 +46,87 @@ func TestCacheTableBasicScan(t *testing.T) { "(10, 110, 1010), (12, 112, 1012), (14, 114, 1014), (16, 116, 1016), (18, 118, 1018)", ) tk.MustExec("alter table tmp1 cache") - assertSelect := func() { - // For TableReader - // First read will read from original table - tk.MustQuery("select * from tmp1 where id>3 order by id").Check(testkit.Rows( + + // For TableReader + // First read will read from original table + tk.MustQuery("select * from tmp1 where id>3 order by id").Check(testkit.Rows( + "5 105 1005", "7 117 1007", "9 109 1009", + "10 110 1010", "12 112 1012", "14 114 1014", "16 116 1016", "18 118 1018", + )) + // Test for join two cache table + tk.MustExec("drop table if exists join_t1, join_t2, join_t3") + tk.MustExec("create table join_t1 (id int)") + tk.MustExec("insert into join_t1 values(1)") + tk.MustExec("alter table join_t1 cache") + tk.MustQuery("select *from join_t1").Check(testkit.Rows("1")) + tk.MustExec("create table join_t2 (id int)") + tk.MustExec("insert into join_t2 values(2)") + tk.MustExec("alter table join_t2 cache") + tk.MustQuery("select *from join_t2").Check(testkit.Rows("2")) + tk.MustExec("create table join_t3 (id int)") + tk.MustExec("insert into join_t3 values(3)") + planUsed := false + for i := 0; i < 10; i++ { + tk.MustQuery("select *from join_t1 join join_t2").Check(testkit.Rows("1 2")) + if lastReadFromCache(tk) { + planUsed = true + break + } + } + require.True(t, planUsed) + + // Test for join a cache table and a normal table + for i := 0; i < 10; i++ { + tk.MustQuery("select * from join_t1 join join_t3").Check(testkit.Rows("1 3")) + if lastReadFromCache(tk) { + // if tk.HasPlan("select *from join_t1 join join_t3", "UnionScan") { + planUsed = true + break + } + } + require.True(t, planUsed) + + // Second read will from cache table + for i := 0; i < 100; i++ { + tk.MustQuery("select * from tmp1 where id>4 order by id").Check(testkit.Rows( "5 105 1005", "7 117 1007", "9 109 1009", "10 110 1010", "12 112 1012", "14 114 1014", "16 116 1016", "18 118 1018", )) - // Test for join two cache table - tk.MustExec("drop table if exists join_t1, join_t2, join_t3") - tk.MustExec("create table join_t1 (id int)") - tk.MustExec("insert into join_t1 values(1)") - tk.MustExec("alter table join_t1 cache") - tk.MustQuery("select *from join_t1").Check(testkit.Rows("1")) - tk.MustExec("create table join_t2 (id int)") - tk.MustExec("insert into join_t2 values(2)") - tk.MustExec("alter table join_t2 cache") - tk.MustQuery("select *from join_t2").Check(testkit.Rows("2")) - tk.MustExec("create table join_t3 (id int)") - tk.MustExec("insert into join_t3 values(3)") - planUsed := false - for i := 0; i < 10; i++ { - tk.MustQuery("select *from join_t1 join join_t2").Check(testkit.Rows("1 2")) - if tk.HasPlan("select *from join_t1 join join_t2", "UnionScan") { - planUsed = true - break - } - } - require.True(t, planUsed) - result := tk.MustQuery("explain format = 'brief' select *from join_t1 join join_t2") - result.Check(testkit.Rows( - "HashJoin 100000000.00 root CARTESIAN inner join", - "├─UnionScan(Build) 10000.00 root ", - "│ └─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:join_t2 keep order:false, stats:pseudo", - "└─UnionScan(Probe) 10000.00 root ", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:join_t1 keep order:false, stats:pseudo")) - // Test for join a cache table and a normal table - for i := 0; i < 10; i++ { - tk.MustQuery("select *from join_t1 join join_t3").Check(testkit.Rows("1 3")) - if tk.HasPlan("select *from join_t1 join join_t3", "UnionScan") { - planUsed = true - break - } - } - require.True(t, planUsed) - result = tk.MustQuery("explain format = 'brief' select *from join_t1 join join_t3") - result.Check(testkit.Rows( - "Projection 100000000.00 root test.join_t1.id, test.join_t3.id", - "└─HashJoin 100000000.00 root CARTESIAN inner join", - " ├─UnionScan(Build) 10000.00 root ", - " │ └─TableReader 10000.00 root data:TableFullScan", - " │ └─TableFullScan 10000.00 cop[tikv] table:join_t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:join_t3 keep order:false, stats:pseudo")) - - // Second read will from cache table - for i := 0; i < 100; i++ { - tk.MustQuery("select * from tmp1 where id>4 order by id").Check(testkit.Rows( - "5 105 1005", "7 117 1007", "9 109 1009", - "10 110 1010", "12 112 1012", "14 114 1014", "16 116 1016", "18 118 1018", - )) - if tk.HasPlan("select * from tmp1 where id>4 order by id", "UnionScan") { - planUsed = true - break - } - } - require.True(t, planUsed) - result = tk.MustQuery("explain format = 'brief' select * from tmp1 where id>4 order by id") - result.Check(testkit.Rows("UnionScan 3333.33 root gt(test.tmp1.id, 4)", - "└─TableReader 3333.33 root data:TableRangeScan", - " └─TableRangeScan 3333.33 cop[tikv] table:tmp1 range:(4,+inf], keep order:true, stats:pseudo")) - // For IndexLookUpReader - for i := 0; i < 10; i++ { - tk.MustQuery("select /*+ use_index(tmp1, u) */ * from tmp1 where u>101 order by u").Check(testkit.Rows( - "5 105 1005", "9 109 1009", "10 110 1010", - "12 112 1012", "3 113 1003", "14 114 1014", "16 116 1016", "7 117 1007", "18 118 1018", - )) - if tk.HasPlan("select /*+ use_index(tmp1, u) */ * from tmp1 where u>101 order by u", "UnionScan") { - planUsed = true - break - } + if lastReadFromCache(tk) { + // if tk.HasPlan("select * from tmp1 where id>4 order by id", "UnionScan") { + planUsed = true + break } - require.True(t, planUsed) - result = tk.MustQuery("explain format = 'brief' select /*+ use_index(tmp1, u) */ * from tmp1 where u>101 order by u") - result.Check(testkit.Rows("UnionScan 3333.33 root gt(test.tmp1.u, 101)", - "└─IndexLookUp 3333.33 root ", - " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:tmp1, index:u(u) range:(101,+inf], keep order:true, stats:pseudo", - " └─TableRowIDScan(Probe) 3333.33 cop[tikv] table:tmp1 keep order:false, stats:pseudo")) - tk.MustQuery("show warnings").Check(testkit.Rows()) - - // For IndexReader - tk.MustQuery("select /*+ use_index(tmp1, u) */ id,u from tmp1 where u>101 order by id").Check(testkit.Rows( - "3 113", "5 105", "7 117", "9 109", "10 110", - "12 112", "14 114", "16 116", "18 118", - )) - tk.MustQuery("show warnings").Check(testkit.Rows()) + } + require.True(t, planUsed) - // For IndexMerge, cache table should not use index merge - tk.MustQuery("select /*+ use_index_merge(tmp1, primary, u) */ * from tmp1 where id>5 or u>110 order by u").Check(testkit.Rows( - "9 109 1009", "10 110 1010", + // For IndexLookUpReader + for i := 0; i < 10; i++ { + tk.MustQuery("select /*+ use_index(tmp1, u) */ * from tmp1 where u>101 order by u").Check(testkit.Rows( + "5 105 1005", "9 109 1009", "10 110 1010", "12 112 1012", "3 113 1003", "14 114 1014", "16 116 1016", "7 117 1007", "18 118 1018", )) - - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 IndexMerge is inapplicable or disabled. Cannot use IndexMerge on TableCache.")) + if lastReadFromCache(tk) { + planUsed = true + break + } } - assertSelect() - + require.True(t, planUsed) + + // For IndexReader + tk.MustQuery("select /*+ use_index(tmp1, u) */ id,u from tmp1 where u>101 order by id").Check(testkit.Rows( + "3 113", "5 105", "7 117", "9 109", "10 110", + "12 112", "14 114", "16 116", "18 118", + )) + tk.MustQuery("show warnings").Check(testkit.Rows()) + + // For IndexMerge, cache table should not use index merge + tk.MustQuery("select /*+ use_index_merge(tmp1, primary, u) */ * from tmp1 where id>5 or u>110 order by u").Check(testkit.Rows( + "9 109 1009", "10 110 1010", + "12 112 1012", "3 113 1003", "14 114 1014", "16 116 1016", "7 117 1007", "18 118 1018", + )) + + tk.MustQuery("show warnings").Check(testkit.Rows()) } func TestCacheCondition(t *testing.T) { @@ -160,44 +139,51 @@ func TestCacheCondition(t *testing.T) { tk.MustExec("alter table t2 cache") // Explain should not trigger cache. - tk.MustQuery("explain select * from t2") for i := 0; i < 10; i++ { + tk.MustQuery("explain select * from t2") time.Sleep(100 * time.Millisecond) - require.False(t, tk.HasPlan("select * from t2 where id>0", "UnionScan")) + require.False(t, lastReadFromCache(tk)) } // Insert should not trigger cache. tk.MustExec("insert into t2 values (1,1)") for i := 0; i < 10; i++ { time.Sleep(100 * time.Millisecond) - require.False(t, tk.HasPlan("select * from t2 where id>0", "UnionScan")) + require.False(t, lastReadFromCache(tk)) } // Update should not trigger cache. tk.MustExec("update t2 set v = v + 1 where id > 0") for i := 0; i < 10; i++ { time.Sleep(100 * time.Millisecond) - require.False(t, tk.HasPlan("select * from t2 where id>0", "UnionScan")) + require.False(t, lastReadFromCache(tk)) } // Contains PointGet Update should not trigger cache. tk.MustExec("update t2 set v = v + 1 where id = 2") for i := 0; i < 10; i++ { time.Sleep(100 * time.Millisecond) - require.False(t, tk.HasPlan("select * from t2 where id>0", "UnionScan")) + require.False(t, lastReadFromCache(tk)) } // Contains PointGet Delete should not trigger cache. tk.MustExec("delete from t2 where id = 2") for i := 0; i < 10; i++ { time.Sleep(100 * time.Millisecond) - require.False(t, tk.HasPlan("select * from t2 where id>0", "UnionScan")) + require.False(t, lastReadFromCache(tk)) } // Normal query should trigger cache. tk.MustQuery("select * from t2") - for !tk.HasPlan("select * from t2 where id>0", "UnionScan") { - tk.MustExec("select * from t2") + cacheUsed := false + for i := 0; i < 100; i++ { + tk.MustQuery("select * from t2") + if lastReadFromCache(tk) { + cacheUsed = true + break + } + time.Sleep(100 * time.Millisecond) } + require.True(t, cacheUsed) } func TestCacheTableBasicReadAndWrite(t *testing.T) { @@ -215,31 +201,38 @@ func TestCacheTableBasicReadAndWrite(t *testing.T) { tk.MustExec("alter table write_tmp1 cache") // Read and add read lock - tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", - "3 113 1003")) + tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", "3 113 1003")) // read lock should valid var i int for i = 0; i < 10; i++ { - if tk.HasPlan("select * from write_tmp1", "UnionScan") { + if lastReadFromCache(tk) { break } // Wait for the cache to be loaded. time.Sleep(50 * time.Millisecond) + tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", "3 113 1003")) } require.True(t, i < 10) tk.MustExec("use test") tk1.MustExec("insert into write_tmp1 values (2, 222, 222)") // write lock exists - require.False(t, tk.HasPlan("select * from write_tmp1", "UnionScan")) + tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", + "2 222 222", + "3 113 1003")) + require.False(t, lastReadFromCache(tk)) + // wait write lock expire and check cache can be used again - for !tk.HasPlan("select * from write_tmp1", "UnionScan") { - tk.MustExec("select * from write_tmp1") + for !lastReadFromCache(tk) { + tk.MustQuery("select * from write_tmp1").Check(testkit.Rows( + "1 101 1001", + "2 222 222", + "3 113 1003")) } tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", "2 222 222", "3 113 1003")) tk1.MustExec("update write_tmp1 set v = 3333 where id = 2") - for !tk.HasPlan("select * from write_tmp1", "UnionScan") { - tk.MustExec("select * from write_tmp1") + for !lastReadFromCache(tk) { + tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", "2 222 222", "3 113 1003")) } tk.MustQuery("select * from write_tmp1").Check(testkit.Rows("1 101 1001", "2 222 3333", "3 113 1003")) } @@ -258,7 +251,8 @@ func TestCacheTableComplexRead(t *testing.T) { var i int for i = 0; i < 100; i++ { time.Sleep(100 * time.Millisecond) - if tk1.HasPlan("select * from complex_cache where id > 7", "UnionScan") { + tk1.MustQuery("select * from complex_cache where id > 7").Check(testkit.Rows("9 109 1009")) + if lastReadFromCache(tk1) { break } } @@ -269,14 +263,16 @@ func TestCacheTableComplexRead(t *testing.T) { tk2.MustQuery("select * from complex_cache where id > 7").Check(testkit.Rows("9 109 1009")) for i = 0; i < 10; i++ { time.Sleep(100 * time.Millisecond) - if tk2.HasPlan("select * from complex_cache where id > 7", "UnionScan") { + tk2.MustQuery("select * from complex_cache where id > 7").Check(testkit.Rows("9 109 1009")) + if lastReadFromCache(tk2) { break } } require.True(t, i < 10) tk2.MustExec("commit") - tk1.HasPlan("select * from complex_cache where id > 7", "UnionScan") + tk1.MustQuery("select * from complex_cache where id > 7").Check(testkit.Rows("9 109 1009")) + require.True(t, lastReadFromCache(tk1)) tk1.MustExec("commit") } @@ -298,7 +294,8 @@ func TestBeginSleepABA(t *testing.T) { tk1.MustQuery("select * from aba").Check(testkit.Rows("1 1")) cacheUsed := false for i := 0; i < 100; i++ { - if tk1.HasPlan("select * from aba", "UnionScan") { + tk1.MustQuery("select * from aba").Check(testkit.Rows("1 1")) + if lastReadFromCache(tk1) { cacheUsed = true break } @@ -309,7 +306,8 @@ func TestBeginSleepABA(t *testing.T) { tk1.MustExec("begin") cacheUsed = false for i := 0; i < 100; i++ { - if tk1.HasPlan("select * from aba", "UnionScan") { + tk1.MustQuery("select * from aba").Check(testkit.Rows("1 1")) + if lastReadFromCache(tk1) { cacheUsed = true break } @@ -323,7 +321,7 @@ func TestBeginSleepABA(t *testing.T) { cacheUsed = false for i := 0; i < 100; i++ { tk2.MustQuery("select * from aba").Check(testkit.Rows("1 2")) - if tk2.HasPlan("select * from aba", "UnionScan") { + if lastReadFromCache(tk2) { cacheUsed = true break } @@ -332,7 +330,8 @@ func TestBeginSleepABA(t *testing.T) { require.True(t, cacheUsed) // tk1 should not use the staled cache, because the data is changed. - require.False(t, tk1.HasPlan("select * from aba", "UnionScan")) + tk1.MustQuery("select * from aba").Check(testkit.Rows("1 1")) + require.False(t, lastReadFromCache(tk1)) } func TestCacheTablePointGet(t *testing.T) { @@ -469,13 +468,13 @@ func TestCacheTableWriteOperatorWaitLockLease(t *testing.T) { tk.MustExec("drop table if exists wait_tb1") tk.MustExec("create table wait_tb1(id int)") tk.MustExec("alter table wait_tb1 cache") - tk.MustExec("select *from wait_tb1") var i int for i = 0; i < 10; i++ { - time.Sleep(100 * time.Millisecond) - if tk.HasPlan("select *from wait_tb1", "UnionScan") { + tk.MustQuery("select * from wait_tb1").Check(testkit.Rows()) + if lastReadFromCache(tk) { break } + time.Sleep(100 * time.Millisecond) } require.True(t, i < 10) stmtsummary.StmtSummaryByDigestMap.Clear() @@ -511,10 +510,10 @@ func TestTableCacheLeaseVariable(t *testing.T) { tk.MustExec(`insert into test_lease_variable(c0, c1, c2) values (1, null, 'green');`) tk.MustExec(`alter table test_lease_variable cache;`) - tk.MustQuery("select * from test_lease_variable").Check(testkit.Rows("1 green")) cached := false for i := 0; i < 20; i++ { - if tk.HasPlan("select * from test_lease_variable", "UnionScan") { + tk.MustQuery("select * from test_lease_variable").Check(testkit.Rows("1 green")) + if lastReadFromCache(tk) { cached = true break } @@ -530,3 +529,62 @@ func TestTableCacheLeaseVariable(t *testing.T) { require.True(t, duration > time.Second) require.True(t, duration < 3*time.Second) } + +func TestMetrics(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists test_metrics;") + tk.MustExec(`create table test_metrics(c0 int, c1 varchar(20), c2 varchar(20), unique key uk(c0));`) + tk.MustExec(`create table nt (c0 int, c1 varchar(20), c2 varchar(20), unique key uk(c0));`) + tk.MustExec(`insert into test_metrics(c0, c1, c2) values (1, null, 'green');`) + tk.MustExec(`alter table test_metrics cache;`) + + tk.MustQuery("select * from test_metrics").Check(testkit.Rows("1 green")) + cached := false + for i := 0; i < 20; i++ { + if lastReadFromCache(tk) { + cached = true + break + } + time.Sleep(50 * time.Millisecond) + tk.MustQuery("select * from test_metrics").Check(testkit.Rows("1 green")) + } + require.True(t, cached) + + counter := metrics.ReadFromTableCacheCounter + pb := &dto.Metric{} + + queries := []string{ + // Table scan + "select * from test_metrics", + // Index scan + "select c0 from test_metrics use index(uk) where c0 > 1", + // Index Lookup + "select c1 from test_metrics use index(uk) where c0 = 1", + // Point Get + "select c0 from test_metrics use index(uk) where c0 = 1", + // // Aggregation + "select count(*) from test_metrics", + // Join + "select * from test_metrics as a join test_metrics as b on a.c0 = b.c0 where a.c1 != 'xxx'", + } + counter.Write(pb) + i := pb.GetCounter().GetValue() + + for _, query := range queries { + tk.MustQuery(query) + i++ + counter.Write(pb) + hit := pb.GetCounter().GetValue() + require.Equal(t, i, hit) + } + + // A counter-example that doesn't increase metrics.ReadFromTableCacheCounter. + tk.MustQuery("select * from nt") + counter.Write(pb) + hit := pb.GetCounter().GetValue() + require.Equal(t, i, hit) +} diff --git a/table/tables/index.go b/table/tables/index.go index 8350925fe15b4..b3d6b9e820868 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -16,12 +16,9 @@ package tables import ( "context" - "io" "sync" - "time" "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -33,54 +30,6 @@ import ( "github.com/pingcap/tidb/util/rowcodec" ) -// indexIter is for KV store index iterator. -type indexIter struct { - it kv.Iterator - idx *index - prefix kv.Key - colInfos []rowcodec.ColInfo - tps []*types.FieldType -} - -// Close does the clean up works when KV store index iterator is closed. -func (c *indexIter) Close() { - if c.it != nil { - c.it.Close() - c.it = nil - } -} - -// Next returns current key and moves iterator to the next step. -func (c *indexIter) Next() (indexData []types.Datum, h kv.Handle, err error) { - if !c.it.Valid() { - return nil, nil, errors.Trace(io.EOF) - } - if !c.it.Key().HasPrefix(c.prefix) { - return nil, nil, errors.Trace(io.EOF) - } - vals, err := tablecodec.DecodeIndexKV(c.it.Key(), c.it.Value(), len(c.colInfos), tablecodec.HandleNotNeeded, c.colInfos) - if err != nil { - return nil, nil, errors.Trace(err) - } - handle, err := tablecodec.DecodeIndexHandle(c.it.Key(), c.it.Value(), len(c.colInfos)) - if err != nil { - return nil, nil, errors.Trace(err) - } - for i, v := range vals { - d, err := tablecodec.DecodeColumnValue(v, c.tps[i], time.Local) - if err != nil { - return nil, nil, errors.Trace(err) - } - indexData = append(indexData, d) - } - // update new iter to next - err = c.it.Next() - if err != nil { - return nil, nil, err - } - return indexData, handle, nil -} - // index is the data structure for index data in the KV store. type index struct { idxInfo *model.IndexInfo @@ -198,8 +147,20 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } + opt.IgnoreAssertion = opt.IgnoreAssertion || c.idxInfo.State != model.StatePublic + if !distinct || skipCheck || opt.Untouched { err = txn.GetMemBuffer().Set(key, idxVal) + if err != nil { + return nil, err + } + if !opt.IgnoreAssertion && (!opt.Untouched) { + if sctx.GetSessionVars().LazyCheckKeyNotExists() && !txn.IsPessimistic() { + err = txn.SetAssertion(key, kv.SetAssertUnknown) + } else { + err = txn.SetAssertion(key, kv.SetAssertNotExist) + } + } return nil, err } @@ -226,11 +187,23 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } if err != nil || len(value) == 0 { - if sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil { + lazyCheck := sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil + if lazyCheck { err = txn.GetMemBuffer().SetWithFlags(key, idxVal, kv.SetPresumeKeyNotExists) } else { err = txn.GetMemBuffer().Set(key, idxVal) } + if err != nil { + return nil, err + } + if opt.IgnoreAssertion { + return nil, nil + } + if lazyCheck && !txn.IsPessimistic() { + err = txn.SetAssertion(key, kv.SetAssertUnknown) + } else { + err = txn.SetAssertion(key, kv.SetAssertNotExist) + } return nil, err } @@ -252,66 +225,14 @@ func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexed } else { err = txn.GetMemBuffer().Delete(key) } - return err -} - -// Drop removes the KV index from store. -func (c *index) Drop(txn kv.Transaction) error { - it, err := txn.Iter(c.prefix, c.prefix.PrefixNext()) if err != nil { return err } - defer it.Close() - - // remove all indices - for it.Valid() { - if !it.Key().HasPrefix(c.prefix) { - break - } - err := txn.GetMemBuffer().Delete(it.Key()) - if err != nil { - return err - } - err = it.Next() - if err != nil { - return err - } - } - return nil -} - -// Seek searches KV index for the entry with indexedValues. -func (c *index) Seek(sc *stmtctx.StatementContext, r kv.Retriever, indexedValues []types.Datum) (iter table.IndexIterator, hit bool, err error) { - key, _, err := c.GenIndexKey(sc, indexedValues, nil, nil) - if err != nil { - return nil, false, err - } - - upperBound := c.prefix.PrefixNext() - it, err := r.Iter(key, upperBound) - if err != nil { - return nil, false, err - } - // check if hit - hit = false - if it.Valid() && it.Key().Cmp(key) == 0 { - hit = true + if c.idxInfo.State == model.StatePublic { + // If the index is in public state, delete this index means it must exists. + err = txn.SetAssertion(key, kv.SetAssertExist) } - colInfos := BuildRowcodecColInfoForIndexColumns(c.idxInfo, c.tblInfo) - tps := BuildFieldTypesForIndexColumns(c.idxInfo, c.tblInfo) - return &indexIter{it: it, idx: c, prefix: c.prefix, colInfos: colInfos, tps: tps}, hit, nil -} - -// SeekFirst returns an iterator which points to the first entry of the KV index. -func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) { - upperBound := c.prefix.PrefixNext() - it, err := r.Iter(c.prefix, upperBound) - if err != nil { - return nil, err - } - colInfos := BuildRowcodecColInfoForIndexColumns(c.idxInfo, c.tblInfo) - tps := BuildFieldTypesForIndexColumns(c.idxInfo, c.tblInfo) - return &indexIter{it: it, idx: c, prefix: c.prefix, colInfos: colInfos, tps: tps}, nil + return err } func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 3e70c85365bd8..a969b6c6a485f 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -16,244 +16,26 @@ package tables_test import ( "context" - "io" "testing" - "time" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/parser/terror" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/rowcodec" "github.com/stretchr/testify/require" ) -func TestIndex(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test ununiq index. - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - values := types.MakeDatums(1, 2) - mockCtx := mock.NewContext() - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - it, err := index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err := it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) - require.NoError(t, err) - require.False(t, exist) - - exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.True(t, exist) - - err = index.Delete(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(0), nil) - require.NoError(t, err) - - _, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, hit, err := index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - err = index.Drop(txn) - require.NoError(t, err) - - it, hit, err = index.Seek(sc, txn, values) - require.NoError(t, err) - require.False(t, hit) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - _, _, err = it.Next() - require.Truef(t, terror.ErrorEqual(err, io.EOF), "err %v", err) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - tblInfo = &model.TableInfo{ - ID: 2, - Indices: []*model.IndexInfo{ - { - ID: 3, - Name: model.NewCIStr("test"), - Unique: true, - Columns: []*model.IndexColumn{ - {Offset: 0}, - {Offset: 1}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {ID: 1, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeVarchar)}, - {ID: 2, Name: model.NewCIStr("c2"), State: model.StatePublic, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeString)}, - }, - } - index = tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - // Test uniq index. - txn, err = store.Begin() - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(2), nil) - require.NotNil(t, err) - - it, err = index.SeekFirst(txn) - require.NoError(t, err) - - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, int64(1), getValues[0].GetInt64()) - require.Equal(t, int64(2), getValues[1].GetInt64()) - require.Equal(t, int64(1), h.IntValue()) - it.Close() - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) - require.NotNil(t, err) - require.Equal(t, int64(1), h.IntValue()) - require.True(t, exist) - - err = txn.Commit(context.Background()) - require.NoError(t, err) - - _, err = index.FetchValues(make([]types.Datum, 0), nil) - require.NotNil(t, err) - - txn, err = store.Begin() - require.NoError(t, err) - - // Test the function of Next when the value of unique key is nil. - values2 := types.MakeDatums(nil, nil) - _, err = index.Create(mockCtx, txn, values2, kv.IntHandle(2), nil) - require.NoError(t, err) - it, err = index.SeekFirst(txn) - require.NoError(t, err) - getValues, h, err = it.Next() - require.NoError(t, err) - require.Len(t, getValues, 2) - require.Equal(t, nil, getValues[0].GetInterface()) - require.Equal(t, nil, getValues[1].GetInterface()) - require.Equal(t, int64(2), h.IntValue()) - it.Close() - - err = txn.Commit(context.Background()) - require.NoError(t, err) -} - -func TestCombineIndexSeek(t *testing.T) { - tblInfo := &model.TableInfo{ - ID: 1, - Indices: []*model.IndexInfo{ - { - ID: 2, - Name: model.NewCIStr("test"), - Columns: []*model.IndexColumn{ - {Offset: 1}, - {Offset: 2}, - }, - }, - }, - Columns: []*model.ColumnInfo{ - {Offset: 0}, - {Offset: 1}, - {Offset: 2}, - }, - } - index := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - - store, clean := testkit.CreateMockStore(t) - defer clean() - txn, err := store.Begin() - require.NoError(t, err) - - mockCtx := mock.NewContext() - values := types.MakeDatums("abc", "def") - _, err = index.Create(mockCtx, txn, values, kv.IntHandle(1), nil) - require.NoError(t, err) - - index2 := tables.NewIndex(tblInfo.ID, tblInfo, tblInfo.Indices[0]) - sc := &stmtctx.StatementContext{TimeZone: time.Local} - iter, hit, err := index2.Seek(sc, txn, types.MakeDatums("abc", nil)) - require.NoError(t, err) - defer iter.Close() - require.False(t, hit) - _, h, err := iter.Next() - require.NoError(t, err) - require.Equal(t, int64(1), h.IntValue()) -} - func TestMultiColumnCommonHandle(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) tblInfo := buildTableInfo(t, "create table t (a int, b int, u varchar(64) unique, nu varchar(64), primary key (a, b), index nu (nu))") var idxUnique, idxNonUnique table.Index for _, idxInfo := range tblInfo.Indices { @@ -389,3 +171,35 @@ func buildTableInfo(t *testing.T, sql string) *model.TableInfo { require.NoError(t, err) return tblInfo } + +func TestIssue29520(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_mutation_checker=1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c year, PRIMARY KEY (c) CLUSTERED, KEY i1(c))") + tk.MustExec("insert into t values('2020')") +} + +func TestAssertionWithLazyCheck(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_txn_assertion_level = 'STRICT'") + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int primary key, v1 int, v2 int, index (v1), unique index (v2))") + tk.MustExec("set @@tidb_constraint_check_in_place = true") + tk.MustExec("insert into t values (1, 1, 1)") + tk.MustGetErrCode("insert into t values (2, 1, 1)", errno.ErrDupEntry) + + tk.MustExec("set @@tidb_constraint_check_in_place = false") + tk.MustExec("insert into t values (3, 3, 3)") + // The constraint check (index key must not exist) will be done while prewriting. TiDB should avoid setting + // assertion on the index key. Even it's set, TiKV will skip checking assertion for mutation types `Insert` and + // `CheckNotExist`. Anyway there should never be assertion failure. + tk.MustGetErrCode("insert into t values (4, 3, 3)", errno.ErrDupEntry) +} diff --git a/table/tables/main_test.go b/table/tables/main_test.go index 9150776cd9404..8ae700fde5223 100644 --- a/table/tables/main_test.go +++ b/table/tables/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/table/tables/mutation_checker.go b/table/tables/mutation_checker.go new file mode 100644 index 0000000000000..0749abbb9cd09 --- /dev/null +++ b/table/tables/mutation_checker.go @@ -0,0 +1,383 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tables + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/rowcodec" + "go.uber.org/zap" +) + +var ( + // ErrInconsistentRowValue is the error when values in a row insertion does not match the expected ones. + ErrInconsistentRowValue = dbterror.ClassTable.NewStd(errno.ErrInconsistentRowValue) + // ErrInconsistentHandle is the error when the handle in the row/index insertions does not match. + ErrInconsistentHandle = dbterror.ClassTable.NewStd(errno.ErrInconsistentHandle) + // ErrInconsistentIndexedValue is the error when decoded values from the index mutation cannot match row value + ErrInconsistentIndexedValue = dbterror.ClassTable.NewStd(errno.ErrInconsistentIndexedValue) +) + +type mutation struct { + key kv.Key + flags kv.KeyFlags + value []byte + indexID int64 // only for index mutations +} + +type columnMaps struct { + ColumnIDToInfo map[int64]*model.ColumnInfo + ColumnIDToFieldType map[int64]*types.FieldType + IndexIDToInfo map[int64]*model.IndexInfo + IndexIDToRowColInfos map[int64][]rowcodec.ColInfo +} + +// CheckDataConsistency checks whether the given set of mutations corresponding to a single row is consistent. +// Namely, assume the database is consistent before, applying the mutations shouldn't break the consistency. +// It aims at reducing bugs that will corrupt data, and preventing mistakes from spreading if possible. +// +// 3 conditions are checked: +// (1) row.value is consistent with input data +// (2) the handle is consistent in row and index insertions +// (3) the keys of the indices are consistent with the values of rows +// +// The check doesn't work and just returns nil when: +// (1) the table is partitioned +// (2) new collation is enabled and restored data is needed +// +// The check is performed on almost every write. Its performance matters. +// Let M = the number of mutations, C = the number of columns in the table, +// I = the sum of the number of columns in all indices, +// The time complexity is O(M * C + I) +// The space complexity is O(M + C + I) +func CheckDataConsistency( + txn kv.Transaction, sessVars *variable.SessionVars, t *TableCommon, + rowToInsert, rowToRemove []types.Datum, memBuffer kv.MemBuffer, sh kv.StagingHandle, +) error { + if t.Meta().GetPartitionInfo() != nil { + return nil + } + if sh == 0 { + // some implementations of MemBuffer doesn't support staging, e.g. that in br/pkg/lightning/backend/kv + return nil + } + indexMutations, rowInsertion, err := collectTableMutationsFromBufferStage(t, memBuffer, sh) + if err != nil { + return errors.Trace(err) + } + + columnMaps := getColumnMaps(txn, t) + + if rowToInsert != nil { + if err := checkRowInsertionConsistency( + sessVars, rowToInsert, rowInsertion, columnMaps.ColumnIDToInfo, columnMaps.ColumnIDToFieldType, t.Meta().Name.O, + ); err != nil { + return errors.Trace(err) + } + } + + if err != nil { + return err + } + + if rowInsertion.key != nil { + if err = checkHandleConsistency(rowInsertion, indexMutations, columnMaps.IndexIDToInfo, t.Meta().Name.O); err != nil { + return errors.Trace(err) + } + } + + if err := checkIndexKeys( + sessVars, t, rowToInsert, rowToRemove, indexMutations, columnMaps.IndexIDToInfo, columnMaps.IndexIDToRowColInfos, + ); err != nil { + return errors.Trace(err) + } + return nil +} + +// checkHandleConsistency checks whether the handles, with regard to a single-row change, +// in row insertions and index insertions are consistent. +// A PUT_index implies a PUT_row with the same handle. +// Deletions are not checked since the values of deletions are unknown +func checkHandleConsistency(rowInsertion mutation, indexMutations []mutation, indexIDToInfo map[int64]*model.IndexInfo, tableName string) error { + var insertionHandle kv.Handle + var err error + + if rowInsertion.key == nil { + return nil + } + insertionHandle, err = tablecodec.DecodeRowKey(rowInsertion.key) + if err != nil { + return errors.Trace(err) + } + + for _, m := range indexMutations { + if len(m.value) == 0 { + continue + } + + indexInfo, ok := indexIDToInfo[m.indexID] + if !ok { + return errors.New("index not found") + } + + indexHandle, err := tablecodec.DecodeIndexHandle(m.key, m.value, len(indexInfo.Columns)) + if err != nil { + return errors.Trace(err) + } + // NOTE: handle type can be different, see issue 29520 + if indexHandle.IsInt() == insertionHandle.IsInt() && indexHandle.Compare(insertionHandle) != 0 { + err = ErrInconsistentHandle.GenWithStackByArgs(tableName, indexInfo.Name.O, indexHandle, insertionHandle, m, rowInsertion) + logutil.BgLogger().Error("inconsistent handle in index and record insertions", zap.Error(err)) + return err + } + } + + return err +} + +// checkIndexKeys checks whether the decoded data from keys of index mutations are consistent with the expected ones. +// +// How it works: +// +// Assume the set of row values changes from V1 to V2, we check +// (1) V2 - V1 = {added indices} +// (2) V1 - V2 = {deleted indices} +// +// To check (1), we need +// (a) {added indices} is a subset of {needed indices} => each index mutation is consistent with the input/row key/value +// (b) {needed indices} is a subset of {added indices}. The check process would be exactly the same with how we generate +// the mutations, thus ignored. +func checkIndexKeys( + sessVars *variable.SessionVars, t *TableCommon, rowToInsert, rowToRemove []types.Datum, + indexMutations []mutation, indexIDToInfo map[int64]*model.IndexInfo, + indexIDToRowColInfos map[int64][]rowcodec.ColInfo, +) error { + var indexData []types.Datum + for _, m := range indexMutations { + indexInfo, ok := indexIDToInfo[m.indexID] + if !ok { + return errors.New("index not found") + } + rowColInfos, ok := indexIDToRowColInfos[m.indexID] + if !ok { + return errors.New("index not found") + } + + // when we cannot decode the key to get the original value + if len(m.value) == 0 && NeedRestoredData(indexInfo.Columns, t.Meta().Columns) { + continue + } + + decodedIndexValues, err := tablecodec.DecodeIndexKV( + m.key, m.value, len(indexInfo.Columns), tablecodec.HandleNotNeeded, rowColInfos, + ) + if err != nil { + return errors.Trace(err) + } + + // reuse the underlying memory, save an allocation + if indexData == nil { + indexData = make([]types.Datum, 0, len(decodedIndexValues)) + } else { + indexData = indexData[:0] + } + + for i, v := range decodedIndexValues { + fieldType := &t.Columns[indexInfo.Columns[i].Offset].FieldType + datum, err := tablecodec.DecodeColumnValue(v, fieldType, sessVars.Location()) + if err != nil { + return errors.Trace(err) + } + indexData = append(indexData, datum) + } + + if len(m.value) == 0 { + err = compareIndexData(sessVars.StmtCtx, t.Columns, indexData, rowToRemove, indexInfo, t.Meta()) + } else { + err = compareIndexData(sessVars.StmtCtx, t.Columns, indexData, rowToInsert, indexInfo, t.Meta()) + } + if err != nil { + return errors.Trace(err) + } + } + return nil +} + +// checkRowInsertionConsistency checks whether the values of row mutations are consistent with the expected ones +// We only check data added since a deletion of a row doesn't care about its value (and we cannot know it) +func checkRowInsertionConsistency( + sessVars *variable.SessionVars, rowToInsert []types.Datum, rowInsertion mutation, + columnIDToInfo map[int64]*model.ColumnInfo, columnIDToFieldType map[int64]*types.FieldType, tableName string, +) error { + if rowToInsert == nil { + // it's a deletion + return nil + } + + decodedData, err := tablecodec.DecodeRowToDatumMap(rowInsertion.value, columnIDToFieldType, sessVars.Location()) + if err != nil { + return errors.Trace(err) + } + + // NOTE: we cannot check if the decoded values contain all columns since some columns may be skipped. It can even be empty + // Instead, we check that decoded index values are consistent with the input row. + + for columnID, decodedDatum := range decodedData { + inputDatum := rowToInsert[columnIDToInfo[columnID].Offset] + cmp, err := decodedDatum.Compare(sessVars.StmtCtx, &inputDatum, collate.GetCollator(decodedDatum.Collation())) + if err != nil { + return errors.Trace(err) + } + if cmp != 0 { + err = ErrInconsistentRowValue.GenWithStackByArgs(tableName, inputDatum.String(), decodedDatum.String()) + logutil.BgLogger().Error("inconsistent row value in row insertion", zap.Error(err)) + return err + } + } + return nil +} + +// collectTableMutationsFromBufferStage collects mutations of the current table from the mem buffer stage +// It returns: (1) all index mutations (2) the only row insertion +// If there are no row insertions, the 2nd returned value is nil +// If there are multiple row insertions, an error is returned +func collectTableMutationsFromBufferStage(t *TableCommon, memBuffer kv.MemBuffer, sh kv.StagingHandle) ( + []mutation, mutation, error, +) { + indexMutations := make([]mutation, 0) + var rowInsertion mutation + var err error + inspector := func(key kv.Key, flags kv.KeyFlags, data []byte) { + // only check the current table + if tablecodec.DecodeTableID(key) == t.physicalTableID { + m := mutation{key, flags, data, 0} + if rowcodec.IsRowKey(key) { + if len(data) > 0 { + if rowInsertion.key == nil { + rowInsertion = m + } else { + err = errors.Errorf( + "multiple row mutations added/mutated, one = %+v, another = %+v", rowInsertion, m, + ) + } + } + } else { + _, m.indexID, _, err = tablecodec.DecodeIndexKey(m.key) + if err != nil { + err = errors.Trace(err) + } + indexMutations = append(indexMutations, m) + } + } + } + memBuffer.InspectStage(sh, inspector) + return indexMutations, rowInsertion, err +} + +// compareIndexData compares the decoded index data with the input data. +// Returns error if the index data is not a subset of the input data. +func compareIndexData( + sc *stmtctx.StatementContext, cols []*table.Column, indexData, input []types.Datum, indexInfo *model.IndexInfo, + tableInfo *model.TableInfo, +) error { + for i := range indexData { + decodedMutationDatum := indexData[i] + expectedDatum := input[indexInfo.Columns[i].Offset] + + tablecodec.TruncateIndexValue( + &expectedDatum, indexInfo.Columns[i], + cols[indexInfo.Columns[i].Offset].ColumnInfo, + ) + tablecodec.TruncateIndexValue( + &decodedMutationDatum, indexInfo.Columns[i], + cols[indexInfo.Columns[i].Offset].ColumnInfo, + ) + + comparison, err := decodedMutationDatum.Compare(sc, &expectedDatum, collate.GetCollator(decodedMutationDatum.Collation())) + if err != nil { + return errors.Trace(err) + } + + if comparison != 0 { + err = ErrInconsistentIndexedValue.GenWithStackByArgs( + tableInfo.Name.O, indexInfo.Name.O, cols[indexInfo.Columns[i].Offset].ColumnInfo.Name.O, + decodedMutationDatum.String(), expectedDatum.String(), + ) + logutil.BgLogger().Error("inconsistent indexed value in index insertion", zap.Error(err)) + return err + } + } + return nil +} + +// getColumnMaps tries to get the columnMaps from transaction options. If there isn't one, it builds one and stores it. +// It saves redundant computations of the map. +func getColumnMaps(txn kv.Transaction, t *TableCommon) columnMaps { + getter := func() (map[int64]columnMaps, bool) { + m, ok := txn.GetOption(kv.TableToColumnMaps).(map[int64]columnMaps) + return m, ok + } + setter := func(maps map[int64]columnMaps) { + txn.SetOption(kv.TableToColumnMaps, maps) + } + columnMaps := getOrBuildColumnMaps(getter, setter, t) + return columnMaps +} + +// getOrBuildColumnMaps tries to get the columnMaps from some place. If there isn't one, it builds one and stores it. +// It saves redundant computations of the map. +func getOrBuildColumnMaps( + getter func() (map[int64]columnMaps, bool), setter func(map[int64]columnMaps), t *TableCommon, +) columnMaps { + tableMaps, ok := getter() + if !ok || tableMaps == nil { + tableMaps = make(map[int64]columnMaps) + } + maps, ok := tableMaps[t.tableID] + if !ok { + maps = columnMaps{ + make(map[int64]*model.ColumnInfo, len(t.Meta().Columns)), + make(map[int64]*types.FieldType, len(t.Meta().Columns)), + make(map[int64]*model.IndexInfo, len(t.Indices())), + make(map[int64][]rowcodec.ColInfo, len(t.Indices())), + } + + for _, col := range t.Meta().Columns { + maps.ColumnIDToInfo[col.ID] = col + maps.ColumnIDToFieldType[col.ID] = &col.FieldType + } + for _, index := range t.Indices() { + if index.Meta().Primary && t.meta.IsCommonHandle { + continue + } + maps.IndexIDToInfo[index.Meta().ID] = index.Meta() + maps.IndexIDToRowColInfos[index.Meta().ID] = BuildRowcodecColInfoForIndexColumns(index.Meta(), t.Meta()) + } + + tableMaps[t.tableID] = maps + setter(tableMaps) + } + return maps +} diff --git a/table/tables/mutation_checker_test.go b/table/tables/mutation_checker_test.go new file mode 100644 index 0000000000000..4a2bfd0508949 --- /dev/null +++ b/table/tables/mutation_checker_test.go @@ -0,0 +1,344 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tables + +import ( + "fmt" + "testing" + "time" + + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/rowcodec" + "github.com/stretchr/testify/require" +) + +func TestCompareIndexData(t *testing.T) { + // dimensions of the domain of compareIndexData + // 1. table structure, where we only care about column types that influence truncating values + // 2. comparison of row data & index data + + type caseData struct { + indexData []types.Datum + inputData []types.Datum + fts []*types.FieldType + indexLength []int + correct bool + } + + // assume the index is on all columns + testData := []caseData{ + { + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string")}, + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string")}, + []*types.FieldType{types.NewFieldType(mysql.TypeShort), types.NewFieldType(mysql.TypeString)}, + []int{types.UnspecifiedLength, types.UnspecifiedLength}, + true, + }, + { + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string")}, + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string2")}, + []*types.FieldType{types.NewFieldType(mysql.TypeShort), types.NewFieldType(mysql.TypeString)}, + []int{types.UnspecifiedLength, types.UnspecifiedLength}, + false, + }, + { + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string")}, + []types.Datum{types.NewIntDatum(1), types.NewStringDatum("some string2")}, + []*types.FieldType{types.NewFieldType(mysql.TypeShort), types.NewFieldType(mysql.TypeString)}, + []int{types.UnspecifiedLength, 11}, + true, + }, + } + + for caseID, data := range testData { + sc := &stmtctx.StatementContext{} + cols := make([]*table.Column, 0) + indexCols := make([]*model.IndexColumn, 0) + for i, ft := range data.fts { + cols = append(cols, &table.Column{ColumnInfo: &model.ColumnInfo{Name: model.NewCIStr(fmt.Sprintf("c%d", i)), FieldType: *ft}}) + indexCols = append(indexCols, &model.IndexColumn{Offset: i, Length: data.indexLength[i]}) + } + indexInfo := &model.IndexInfo{Name: model.NewCIStr("i0"), Columns: indexCols} + + err := compareIndexData(sc, cols, data.indexData, data.inputData, indexInfo, &model.TableInfo{Name: model.NewCIStr("t")}) + require.Equal(t, data.correct, err == nil, "case id = %v", caseID) + } +} + +func TestCheckRowInsertionConsistency(t *testing.T) { + sessVars := variable.NewSessionVars() + rd := rowcodec.Encoder{Enable: true} + + // mocked data + mockRowKey233 := tablecodec.EncodeRowKeyWithHandle(1, kv.IntHandle(233)) + mockValue233, err := tablecodec.EncodeRow( + sessVars.StmtCtx, []types.Datum{types.NewIntDatum(233)}, []int64{101}, nil, nil, &rd, + ) + require.Nil(t, err) + fakeRowInsertion := mutation{key: []byte{1, 1}, value: []byte{1, 1, 1}} + + type caseData struct { + columnIDToInfo map[int64]*model.ColumnInfo + columnIDToFieldType map[int64]*types.FieldType + rowToInsert []types.Datum + rowInsertion mutation + correct bool + } + + testData := []caseData{ + { + // expected correct behavior + map[int64]*model.ColumnInfo{ + 101: { + ID: 101, + Offset: 0, + FieldType: *types.NewFieldType(mysql.TypeShort), + }, + }, + map[int64]*types.FieldType{ + 101: types.NewFieldType(mysql.TypeShort), + }, + []types.Datum{types.NewIntDatum(233)}, + mutation{key: mockRowKey233, value: mockValue233}, + true, + }, + { + // mismatching mutation + map[int64]*model.ColumnInfo{ + 101: { + ID: 101, + Offset: 0, + FieldType: *types.NewFieldType(mysql.TypeShort), + }, + }, + map[int64]*types.FieldType{ + 101: types.NewFieldType(mysql.TypeShort), + }, + []types.Datum{types.NewIntDatum(1)}, + fakeRowInsertion, + false, + }, + { + // no input row + map[int64]*model.ColumnInfo{}, + map[int64]*types.FieldType{}, + nil, + fakeRowInsertion, + true, + }, + { + // invalid value + map[int64]*model.ColumnInfo{ + 101: { + ID: 101, + Offset: 0, + FieldType: *types.NewFieldType(mysql.TypeShort), + }, + }, + map[int64]*types.FieldType{ + 101: types.NewFieldType(mysql.TypeShort), + }, + []types.Datum{types.NewIntDatum(233)}, + mutation{key: mockRowKey233, value: []byte{0, 1, 2, 3}}, + false, + }, + } + + for caseID, data := range testData { + err := checkRowInsertionConsistency( + sessVars, data.rowToInsert, data.rowInsertion, data.columnIDToInfo, data.columnIDToFieldType, "t", + ) + require.Equal(t, data.correct, err == nil, "case id = %v", caseID) + } +} + +func TestCheckIndexKeysAndCheckHandleConsistency(t *testing.T) { + // dimensions of the domain of checkIndexKeys: + // 1. location *2 + // 2. table structure + // (1) unique index/non-unique index *2 + // (2) clustered index *2 + // (3) string collation *2 + // We don't test primary clustered index and int handle, since they should not have index mutations. + // Assume PK is always the first column (string). + + // cases + locations := []*time.Location{time.UTC, time.Local} + indexInfos := []*model.IndexInfo{ + { + ID: 1, + State: model.StatePublic, + Primary: false, + Unique: true, + Columns: []*model.IndexColumn{ + { + Offset: 1, + Length: types.UnspecifiedLength, + }, + { + Offset: 0, + Length: types.UnspecifiedLength, + }, + }, + }, + { + ID: 2, + State: model.StatePublic, + Primary: false, + Unique: false, + Columns: []*model.IndexColumn{ + { + Offset: 1, + Length: types.UnspecifiedLength, + }, + { + Offset: 0, + Length: types.UnspecifiedLength, + }, + }, + }, + } + columnInfoSets := [][]*model.ColumnInfo{ + { + {ID: 1, Offset: 0, FieldType: *types.NewFieldType(mysql.TypeString)}, + {ID: 2, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeDatetime)}, + }, + { + {ID: 1, Offset: 0, FieldType: *types.NewFieldTypeWithCollation(mysql.TypeString, "utf8_unicode_ci", + types.UnspecifiedLength)}, + {ID: 2, Offset: 1, FieldType: *types.NewFieldType(mysql.TypeDatetime)}, + }, + } + sessVars := variable.NewSessionVars() + rd := rowcodec.Encoder{Enable: true} + + now := types.CurrentTime(mysql.TypeDatetime) + rowToInsert := []types.Datum{ + types.NewStringDatum("some string"), + types.NewTimeDatum(now), + } + anotherTime, err := now.Add(sessVars.StmtCtx, types.NewDuration(24, 0, 0, 0, 0)) + require.Nil(t, err) + rowToRemove := []types.Datum{ + types.NewStringDatum("old string"), + types.NewTimeDatum(anotherTime), + } + + getter := func() (map[int64]columnMaps, bool) { + return nil, false + } + setter := func(maps map[int64]columnMaps) {} + + // test + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + for _, isCommonHandle := range []bool{true, false} { + for _, lc := range locations { + for _, columnInfos := range columnInfoSets { + sessVars.StmtCtx.TimeZone = lc + tableInfo := model.TableInfo{ + ID: 1, + Name: model.NewCIStr("t"), + Columns: columnInfos, + Indices: indexInfos, + PKIsHandle: false, + IsCommonHandle: isCommonHandle, + } + table := MockTableFromMeta(&tableInfo).(*TableCommon) + var handle, corruptedHandle kv.Handle + if isCommonHandle { + encoded, err := codec.EncodeKey(sessVars.StmtCtx, nil, rowToInsert[0]) + require.Nil(t, err) + corrupted := make([]byte, len(encoded)) + copy(corrupted, encoded) + corrupted[len(corrupted)-1] ^= 1 + handle, err = kv.NewCommonHandle(encoded) + require.Nil(t, err) + corruptedHandle, err = kv.NewCommonHandle(corrupted) + require.Nil(t, err) + } else { + handle = kv.IntHandle(1) + corruptedHandle = kv.IntHandle(2) + } + + for i, indexInfo := range indexInfos { + index := table.indices[i] + maps := getOrBuildColumnMaps(getter, setter, table) + + // test checkIndexKeys + insertionKey, insertionValue, err := buildIndexKeyValue(index, rowToInsert, sessVars, tableInfo, + indexInfo, table, handle) + require.Nil(t, err) + deletionKey, _, err := buildIndexKeyValue(index, rowToRemove, sessVars, tableInfo, indexInfo, table, + handle) + require.Nil(t, err) + indexMutations := []mutation{ + {key: insertionKey, value: insertionValue, indexID: indexInfo.ID}, + {key: deletionKey, indexID: indexInfo.ID}, + } + err = checkIndexKeys( + sessVars, table, rowToInsert, rowToRemove, indexMutations, maps.IndexIDToInfo, + maps.IndexIDToRowColInfos, + ) + require.Nil(t, err) + + // test checkHandleConsistency + rowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, handle) + corruptedRowKey := tablecodec.EncodeRowKeyWithHandle(table.tableID, corruptedHandle) + rowValue, err := tablecodec.EncodeRow(sessVars.StmtCtx, rowToInsert, []int64{1, 2}, nil, nil, &rd) + require.Nil(t, err) + rowMutation := mutation{key: rowKey, value: rowValue} + corruptedRowMutation := mutation{key: corruptedRowKey, value: rowValue} + err = checkHandleConsistency(rowMutation, indexMutations, maps.IndexIDToInfo, "t") + require.Nil(t, err) + err = checkHandleConsistency(corruptedRowMutation, indexMutations, maps.IndexIDToInfo, "t") + require.NotNil(t, err) + } + } + } + } +} + +func buildIndexKeyValue(index table.Index, rowToInsert []types.Datum, sessVars *variable.SessionVars, + tableInfo model.TableInfo, indexInfo *model.IndexInfo, table *TableCommon, handle kv.Handle) ([]byte, []byte, error) { + indexedValues, err := index.FetchValues(rowToInsert, nil) + if err != nil { + return nil, nil, err + } + key, distinct, err := tablecodec.GenIndexKey( + sessVars.StmtCtx, &tableInfo, indexInfo, 1, indexedValues, handle, nil, + ) + if err != nil { + return nil, nil, err + } + rsData := TryGetHandleRestoredDataWrapper(table, rowToInsert, nil, indexInfo) + value, err := tablecodec.GenIndexValuePortal( + sessVars.StmtCtx, &tableInfo, indexInfo, NeedRestoredData(indexInfo.Columns, tableInfo.Columns), + distinct, false, indexedValues, handle, 0, rsData, + ) + if err != nil { + return nil, nil, err + } + return key, value, nil +} diff --git a/table/tables/partition.go b/table/tables/partition.go index 71a2db52e81ab..9362b41626045 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -906,6 +906,29 @@ func (t *partitionedTable) PartitionExpr() (*PartitionExpr, error) { return t.partitionExpr, nil } +func (t *partitionedTable) GetPartitionColumnNames() []model.CIStr { + // PARTITION BY {LIST|RANGE} COLUMNS uses columns directly without expressions + pi := t.Meta().Partition + if len(pi.Columns) > 0 { + return pi.Columns + } + + partitionCols := expression.ExtractColumns(t.partitionExpr.Expr) + colIDs := make([]int64, 0, len(partitionCols)) + for _, col := range partitionCols { + colIDs = append(colIDs, col.ID) + } + colNames := make([]model.CIStr, 0, len(partitionCols)) + for _, colID := range colIDs { + for _, col := range t.Cols() { + if col.ID == colID { + colNames = append(colNames, col.Name) + } + } + } + return colNames +} + // PartitionRecordKey is exported for test. func PartitionRecordKey(pid int64, handle int64) kv.Key { recordPrefix := tablecodec.GenTableRecordPrefix(pid) diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index 0d45e4be6518d..38cffa4debed1 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -609,6 +609,69 @@ func TestIssue24746(t *testing.T) { require.True(t, table.ErrRowDoesNotMatchGivenPartitionSet.Equal(err)) } +func TestIssue31629(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_list_partition = 1") + tk.MustExec("create database Issue31629") + defer tk.MustExec("drop database Issue31629") + tk.MustExec("use Issue31629") + // Test following partition types: + // HASH, RANGE, LIST: + // - directly on a single int column + // - with expression on multiple columns + // RANGE/LIST COLUMNS single column + // RANGE/LIST COLUMNS -- Verify that only single column is allowed and no expression + tests := []struct { + create string + fail bool + cols []string + }{ + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by range(col1) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by range(Col1+col3) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by hash(col1) partitions 3", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by hash(Col1+col3) partitions 3", false, []string{"Col1", "col3"}}, + {"(col1 int, col2 varchar(60), col3 int, primary key(col1)) partition by list(col1) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"col1"}}, + {"(Col1 int, col2 varchar(60), col3 int, primary key(Col1,col3)) partition by list(Col1+col3) (partition p0 values in (5,6,7,8,9),partition p1 values in (10,11,12,13,14), partition p2 values in (20,21,22,23,24))", false, []string{"Col1", "col3"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col2,col3) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by range columns (col1+1) (partition p0 values less than (""),partition p1 values less than ("MID"), partition p2 values less than maxvalue)`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, false, []string{"col2"}}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col2,col3) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + {`(col1 int, col2 varchar(60), col3 int, primary key(col2)) partition by list columns (col1+1) (partition p0 values in ("","First"),partition p1 values in ("MID","Middle"), partition p2 values in ("Last","Unknown"))`, true, nil}, + } + + for i, tt := range tests { + + createTable := "create table t1 " + tt.create + res, err := tk.Exec(createTable) + if res != nil { + res.Close() + } + if err != nil { + if tt.fail { + continue + } + } + require.Falsef(t, tt.fail, "test %d succeeded but was expected to fail! %s", i, createTable) + require.NoError(t, err) + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tb, err := dom.InfoSchema().TableByName(model.NewCIStr("Issue31629"), model.NewCIStr("t1")) + require.NoError(t, err) + tbp, ok := tb.(table.PartitionedTable) + require.Truef(t, ok, "test %d does not generate a table.PartitionedTable: %s (%T, %+v)", i, createTable, tb, tb) + colNames := tbp.GetPartitionColumnNames() + checkNames := []model.CIStr{model.NewCIStr(tt.cols[0])} + for i := 1; i < len(tt.cols); i++ { + checkNames = append(checkNames, model.NewCIStr(tt.cols[i])) + } + require.ElementsMatchf(t, colNames, checkNames, "test %d %s", i, createTable) + tk.MustExec("drop table t1") + } +} + func TestIssue31721(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/table/tables/tables.go b/table/tables/tables.go index 716b2e879cbcc..56bbaacf55271 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -28,6 +28,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -429,6 +430,27 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, if err = memBuffer.Set(key, value); err != nil { return err } + + failpoint.Inject("updateRecordForceAssertNotExist", func() { + // Assert the key doesn't exist while it actually exists. This is helpful to test if assertion takes effect. + // Since only the first assertion takes effect, set the injected assertion before setting the correct one to + // override it. + if sctx.GetSessionVars().ConnectionID != 0 { + logutil.BgLogger().Info("[failpoint] force asserting not exist on UpdateRecord", zap.Uint64("startTS", txn.StartTS())) + if err = txn.SetAssertion(key, kv.SetAssertNotExist); err != nil { + failpoint.Return(err) + } + } + }) + if err = txn.SetAssertion(key, kv.SetAssertExist); err != nil { + return err + } + + if sessVars.EnableMutationChecker { + if err = CheckDataConsistency(txn, sessVars, t, newData, oldData, memBuffer, sh); err != nil { + return errors.Trace(err) + } + } memBuffer.Release(sh) if shouldWriteBinlog(sctx, t.meta) { if !t.meta.PKIsHandle && !t.meta.IsCommonHandle { @@ -829,6 +851,26 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . return nil, err } + failpoint.Inject("addRecordForceAssertExist", func() { + // Assert the key exists while it actually doesn't. This is helpful to test if assertion takes effect. + // Since only the first assertion takes effect, set the injected assertion before setting the correct one to + // override it. + if sctx.GetSessionVars().ConnectionID != 0 { + logutil.BgLogger().Info("[failpoint] force asserting exist on AddRecord", zap.Uint64("startTS", txn.StartTS())) + if err = txn.SetAssertion(key, kv.SetAssertExist); err != nil { + failpoint.Return(nil, err) + } + } + }) + if setPresume && !txn.IsPessimistic() { + err = txn.SetAssertion(key, kv.SetAssertUnknown) + } else { + err = txn.SetAssertion(key, kv.SetAssertNotExist) + } + if err != nil { + return nil, err + } + var createIdxOpts []table.CreateIdxOptFunc if len(opts) > 0 { createIdxOpts = make([]table.CreateIdxOptFunc, 0, len(opts)) @@ -844,6 +886,12 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . return h, err } + if sessVars.EnableMutationChecker { + if err = CheckDataConsistency(txn, sessVars, t, r, nil, memBuffer, sh); err != nil { + return nil, errors.Trace(err) + } + } + memBuffer.Release(sh) if shouldWriteBinlog(sctx, t.meta) { @@ -1055,15 +1103,20 @@ func GetChangingColVal(ctx sessionctx.Context, cols []*table.Column, col *table. // RemoveRecord implements table.Table RemoveRecord interface. func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []types.Datum) error { - err := t.removeRowData(ctx, h) + txn, err := ctx.Txn(true) if err != nil { return err } - txn, err := ctx.Txn(true) + memBuffer := txn.GetMemBuffer() + sh := memBuffer.Staging() + defer memBuffer.Cleanup(sh) + + err = t.removeRowData(ctx, h) if err != nil { return err } + if m := t.Meta(); m.TempTableType != model.TempTableNone { if tmpTable := addTemporaryTable(ctx, m); tmpTable != nil { if err := checkTempTableSize(ctx, tmpTable, m); err != nil { @@ -1091,6 +1144,15 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type return err } + sessVars := ctx.GetSessionVars() + sc := sessVars.StmtCtx + if sessVars.EnableMutationChecker { + if err = CheckDataConsistency(txn, sessVars, t, nil, r, memBuffer, sh); err != nil { + return errors.Trace(err) + } + } + memBuffer.Release(sh) + if shouldWriteBinlog(ctx, t.meta) { cols := t.Cols() colIDs := make([]int64, 0, len(cols)+1) @@ -1116,7 +1178,6 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type return nil } colSize := make(map[int64]int64, len(t.Cols())) - sc := ctx.GetSessionVars().StmtCtx for id, col := range t.Cols() { size, err := codec.EstimateValueSize(sc, r[id]) if err != nil { @@ -1206,6 +1267,21 @@ func (t *TableCommon) removeRowData(ctx sessionctx.Context, h kv.Handle) error { } key := t.RecordKey(h) + failpoint.Inject("removeRecordForceAssertNotExist", func() { + // Assert the key doesn't exist while it actually exists. This is helpful to test if assertion takes effect. + // Since only the first assertion takes effect, set the injected assertion before setting the correct one to + // override it. + if ctx.GetSessionVars().ConnectionID != 0 { + logutil.BgLogger().Info("[failpoint] force asserting not exist on RemoveRecord", zap.Uint64("startTS", txn.StartTS())) + if err = txn.SetAssertion(key, kv.SetAssertNotExist); err != nil { + failpoint.Return(err) + } + } + }) + err = txn.SetAssertion(key, kv.SetAssertExist) + if err != nil { + return err + } return txn.Delete(key) } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index bc2d8440f7e04..8f1dc773be5f3 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -16,18 +16,21 @@ package tables_test import ( "context" + "fmt" "math" "strconv" "testing" "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -761,3 +764,192 @@ func TestConstraintCheckForOptimisticUntouched(t *testing.T) { err = tk.ExecToErr("commit") require.Error(t, err) } + +func TestTxnAssertion(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + se, err := session.CreateSession4Test(store) + se.SetConnectionID(1) + require.NoError(t, err) + require.True(t, se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk := testkit.NewTestKit(t, store) + tk.SetSession(se) + + fpAdd := "github.com/pingcap/tidb/table/tables/addRecordForceAssertExist" + fpUpdate := "github.com/pingcap/tidb/table/tables/updateRecordForceAssertNotExist" + fpRemove := "github.com/pingcap/tidb/table/tables/removeRecordForceAssertNotExist" + + runStmtInTxn := func(pessimistic bool, stmts ...string) error { + if pessimistic { + tk.MustExec("begin pessimistic") + } else { + tk.MustExec("begin optimistic") + } + for _, stmt := range stmts { + tk.MustExec(stmt) + } + return tk.ExecToErr("commit") + } + + withFailpoint := func(fp string, f func()) { + require.NoError(t, failpoint.Enable(fp, "return")) + defer func() { + require.NoError(t, failpoint.Disable(fp)) + }() + f() + } + + expectAssertionErr := func(assertionLevel string, err error) { + if assertionLevel == "STRICT" { + require.NotNil(t, err) + require.Contains(t, err.Error(), "assertion failed") + } else { + require.NoError(t, err) + } + } + + testAssertionBasicImpl := func(level string, lock bool, lockIdx bool, useCommonHandle bool) { + tk.MustExec("set @@tidb_txn_assertion_level = " + level) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + if useCommonHandle { + tk.MustExec("create table t(id varchar(64) primary key clustered, v int, v2 int, v3 int, v4 varchar(64), index(v2), unique index(v3), index(v4))") + } else { + tk.MustExec("create table t(id int primary key, v int, v2 int, v3 int, v4 varchar(64), index(v2), unique index(v3), index(v4))") + } + + var id1, id2, id3 interface{} + if useCommonHandle { + id1, id2, id3 = "1", "2", "3" + } else { + id1, id2, id3 = 1, 2, 3 + } + + // Auto commit + tk.MustExec("insert into t values (?, 10, 100, 1000, '10000')", id1) + tk.MustExec("update t set v = v + 1 where id = ?", id1) + tk.MustExec("delete from t where id = 1") + + // Optimistic + tk.MustExec("insert into t values (?, 20, 200, 2000, '20000'), (?, 30, 300, 3000, '30000')", id2, id3) + tk.MustExec("begin optimistic") + if lock { + tk.MustExec("select * from t where id in (?, ?, ?) for update", id1, id2, id3) + } + if lockIdx { + tk.MustExec("select * from t where v3 in (1000, 2000, 3000) for update") + } + tk.MustExec("insert into t values (?, 10, 100, 1000, '10000')", id1) + tk.MustExec("update t set v = v + 1 where id = ?", id2) + tk.MustExec("delete from t where id = ?", id3) + tk.MustExec("commit") + + // Pessimistic + tk.MustExec("delete from t") + tk.MustExec("insert into t values (?, 20, 200, 2000, '20000'), (?, 30, 300, 3000, '30000')", id2, id3) + tk.MustExec("begin pessimistic") + if lock { + tk.MustExec("select * from t where id in (?, ?, ?) for update", id1, id2, id3) + } + if lockIdx { + tk.MustExec("select * from t where v3 in (1000, 2000, 3000) for update") + } + tk.MustExec("insert into t values (?, 10, 100, 1000, '10000')", id1) + tk.MustExec("update t set v = v + 1 where id = ?", id2) + tk.MustExec("delete from t where id = ?", id3) + tk.MustExec("commit") + + // Inject incorrect assertion so that it must fail. + + // Auto commit + tk.MustExec("delete from t") + tk.MustExec("insert into t values (?, 20, 200, 2000, '20000'), (?, 30, 300, 3000, '30000')", id2, id3) + withFailpoint(fpAdd, func() { + err = tk.ExecToErr("insert into t values (?, 10, 100, 1000, '10000')", id1) + expectAssertionErr(level, err) + }) + withFailpoint(fpUpdate, func() { + err = tk.ExecToErr("update t set v = v + 1 where id = ?", id2) + expectAssertionErr(level, err) + }) + withFailpoint(fpRemove, func() { + err = tk.ExecToErr("delete from t where id = ?", id3) + expectAssertionErr(level, err) + }) + + var lockStmts []string = nil + if lock { + lockStmts = append(lockStmts, fmt.Sprintf("select * from t where id in (%#v, %#v, %#v) for update", id1, id2, id3)) + } + if lockIdx { + lockStmts = append(lockStmts, "select * from t where v3 in (1000, 2000, 3000) for update") + } + + // Optimistic + tk.MustExec("delete from t") + tk.MustExec("insert into t values (?, 20, 200, 2000, '20000'), (?, 30, 300, 3000, '30000')", id2, id3) + withFailpoint(fpAdd, func() { + err = runStmtInTxn(false, append(lockStmts, fmt.Sprintf("insert into t values (%#v, 10, 100, 1000, '10000')", id1))...) + expectAssertionErr(level, err) + }) + withFailpoint(fpUpdate, func() { + err = runStmtInTxn(false, append(lockStmts, fmt.Sprintf("update t set v = v + 1 where id = %#v", id2))...) + expectAssertionErr(level, err) + }) + withFailpoint(fpRemove, func() { + err = runStmtInTxn(false, append(lockStmts, fmt.Sprintf("delete from t where id = %#v", id3))...) + expectAssertionErr(level, err) + }) + + // Pessimistic + tk.MustExec("delete from t") + tk.MustExec("insert into t values (?, 20, 200, 2000, '20000'), (?, 30, 300, 3000, '30000')", id2, id3) + withFailpoint(fpAdd, func() { + err = runStmtInTxn(true, append(lockStmts, fmt.Sprintf("insert into t values (%#v, 10, 100, 1000, '10000')", id1))...) + expectAssertionErr(level, err) + }) + withFailpoint(fpUpdate, func() { + err = runStmtInTxn(true, append(lockStmts, fmt.Sprintf("update t set v = v + 1 where id = %#v", id2))...) + expectAssertionErr(level, err) + }) + withFailpoint(fpRemove, func() { + err = runStmtInTxn(true, append(lockStmts, fmt.Sprintf("delete from t where id = %#v", id3))...) + expectAssertionErr(level, err) + }) + } + + for _, level := range []string{"STRICT", "OFF"} { + for _, lock := range []bool{false, true} { + for _, lockIdx := range []bool{false, true} { + for _, useCommonHandle := range []bool{false, true} { + t.Logf("testing testAssertionBasicImpl level: %v, lock: %v, lockIdx: %v, useCommonHandle: %v...", level, lock, lockIdx, useCommonHandle) + testAssertionBasicImpl(level, lock, lockIdx, useCommonHandle) + } + } + } + } + + testUntouchedIndexImpl := func(level string, pessimistic bool) { + tk.MustExec("set @@tidb_txn_assertion_level = " + level) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key, v int, v2 int, v3 int, index(v2), unique index(v3))") + tk.MustExec("insert into t values (1, 10, 100, 1000)") + + if pessimistic { + tk.MustExec("begin pessimistic") + } else { + tk.MustExec("begin optimistic") + } + tk.MustExec("update t set v = v + 1 where id = 1") + tk.MustExec("delete from t where id = 1") + tk.MustExec("insert into t values (1, 11, 101, 1001)") + tk.MustExec("commit") + } + + testUntouchedIndexImpl("STRICT", false) + testUntouchedIndexImpl("STRICT", true) + testUntouchedIndexImpl("OFF", false) + testUntouchedIndexImpl("OFF", true) +} diff --git a/table/temptable/interceptor_test.go b/table/temptable/interceptor_test.go index f75ea1bf0bd6d..79fbb654fe359 100644 --- a/table/temptable/interceptor_test.go +++ b/table/temptable/interceptor_test.go @@ -868,7 +868,7 @@ func TestInterceptorOnBatchGet(t *testing.T) { inter = emptyRetrieverInterceptor } result, err := inter.OnBatchGet(ctx, snap, c.keys) - require.Nil(t, err, i) + require.NoError(t, err, i) require.NotNil(t, result, i) require.Equal(t, c.result, result, i) if c.nilSession { diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index 2b312cecefc9b..ff6bb04f49f77 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -35,7 +35,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 510bbc445a051..54445d3e95e19 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -584,7 +584,7 @@ func Unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type mysql.TypeLong, mysql.TypeLonglong, mysql.TypeDouble: return datum, nil case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - t := types.NewTime(types.ZeroCoreTime, ft.Tp, int8(ft.Decimal)) + t := types.NewTime(types.ZeroCoreTime, ft.Tp, ft.Decimal) var err error err = t.FromPackedUint(datum.GetUint64()) if err != nil { @@ -600,7 +600,7 @@ func Unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type datum.SetMysqlTime(t) return datum, nil case mysql.TypeDuration: // duration should read fsp from column meta data - dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: int8(ft.Decimal)} + dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: ft.Decimal} datum.SetMysqlDuration(dur) return datum, nil case mysql.TypeEnum: diff --git a/telemetry/cte_test/cte_test.go b/telemetry/cte_test/cte_test.go index 5849b285f97e6..6f3e5c735eb3e 100644 --- a/telemetry/cte_test/cte_test.go +++ b/telemetry/cte_test/cte_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/testbridge" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" "go.uber.org/goleak" ) @@ -36,7 +36,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } @@ -48,6 +48,7 @@ func TestCTEPreviewAndReport(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) s := newSuite(t) defer s.close() diff --git a/telemetry/data_cluster_hardware.go b/telemetry/data_cluster_hardware.go index c3b3d716eb682..d357e9243fd0b 100644 --- a/telemetry/data_cluster_hardware.go +++ b/telemetry/data_cluster_hardware.go @@ -69,11 +69,7 @@ func normalizeFieldName(name string) string { func getClusterHardware(ctx sessionctx.Context) ([]*clusterHardwareItem, error) { exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, `SELECT TYPE, INSTANCE, DEVICE_TYPE, DEVICE_NAME, NAME, VALUE FROM information_schema.cluster_hardware`) - if err != nil { - return nil, errors.Trace(err) - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, `SELECT TYPE, INSTANCE, DEVICE_TYPE, DEVICE_NAME, NAME, VALUE FROM information_schema.cluster_hardware`) if err != nil { return nil, errors.Trace(err) } diff --git a/telemetry/data_cluster_info.go b/telemetry/data_cluster_info.go index 7ba04df5d6b9d..40f87bccdfd3d 100644 --- a/telemetry/data_cluster_info.go +++ b/telemetry/data_cluster_info.go @@ -37,11 +37,7 @@ type clusterInfoItem struct { func getClusterInfo(ctx sessionctx.Context) ([]*clusterInfoItem, error) { // Explicitly list all field names instead of using `*` to avoid potential leaking sensitive info when adding new fields in future. exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), true, `SELECT TYPE, INSTANCE, STATUS_ADDRESS, VERSION, GIT_HASH, START_TIME, UPTIME FROM information_schema.cluster_info`) - if err != nil { - return nil, errors.Trace(err) - } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, `SELECT TYPE, INSTANCE, STATUS_ADDRESS, VERSION, GIT_HASH, START_TIME, UPTIME FROM information_schema.cluster_info`) if err != nil { return nil, errors.Trace(err) } diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 486021a5df7d9..521d5a5a3c2f9 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -77,7 +77,7 @@ func getClusterIndexUsageInfo(ctx sessionctx.Context) (cu *ClusterIndexUsage, er exec := ctx.(sqlexec.RestrictedSQLExecutor) // query INFORMATION_SCHEMA.tables to get the latest table information about ClusterIndex - stmt, err := exec.ParseWithParams(context.TODO(), true, ` + rows, _, err := exec.ExecRestrictedSQL(context.TODO(), nil, ` SELECT left(sha2(TABLE_NAME, 256), 6) table_name_hash, TIDB_PK_TYPE, TABLE_SCHEMA, TABLE_NAME FROM information_schema.tables WHERE table_schema not in ('INFORMATION_SCHEMA', 'METRICS_SCHEMA', 'PERFORMANCE_SCHEMA', 'mysql') @@ -86,10 +86,6 @@ func getClusterIndexUsageInfo(ctx sessionctx.Context) (cu *ClusterIndexUsage, er if err != nil { return nil, err } - rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) - if err != nil { - return nil, err - } defer func() { if r := recover(); r != nil { diff --git a/telemetry/id.go b/telemetry/id.go index dac130826596a..057585d48a790 100644 --- a/telemetry/id.go +++ b/telemetry/id.go @@ -19,7 +19,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/telemetry/main_test.go b/telemetry/main_test.go index feb7337e8fac3..41c2c972ebbd8 100644 --- a/telemetry/main_test.go +++ b/telemetry/main_test.go @@ -30,7 +30,7 @@ func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/telemetry/status.go b/telemetry/status.go index b41a59ebbe55d..a5d2aefe6fca6 100644 --- a/telemetry/status.go +++ b/telemetry/status.go @@ -19,7 +19,7 @@ import ( "encoding/json" "github.com/pingcap/errors" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" ) const ( diff --git a/telemetry/telemetry.go b/telemetry/telemetry.go index 481d077087cba..3f383b3d426e8 100644 --- a/telemetry/telemetry.go +++ b/telemetry/telemetry.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) diff --git a/telemetry/telemetry_test.go b/telemetry/telemetry_test.go index 56fc2da76852a..eafccce735efd 100644 --- a/telemetry/telemetry_test.go +++ b/telemetry/telemetry_test.go @@ -25,13 +25,14 @@ import ( "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/integration" + "go.etcd.io/etcd/tests/v3/integration" ) func TestTrackingID(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer etcdCluster.Terminate(t) @@ -53,6 +54,7 @@ func TestPreview(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer etcdCluster.Terminate(t) @@ -106,6 +108,7 @@ func TestReport(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } + integration.BeforeTest(t) etcdCluster := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 1}) defer etcdCluster.Terminate(t) diff --git a/testkit/asynctestkit.go b/testkit/asynctestkit.go index f73d77a6dc03e..34a8d1fdb0dbe 100644 --- a/testkit/asynctestkit.go +++ b/testkit/asynctestkit.go @@ -53,7 +53,7 @@ func NewAsyncTestKit(t *testing.T, store kv.Storage) *AsyncTestKit { // OpenSession opens new session ctx if no exists one and use db. func (tk *AsyncTestKit) OpenSession(ctx context.Context, db string) context.Context { - if tryRetrieveSession(ctx) == nil { + if TryRetrieveSession(ctx) == nil { se, err := session.CreateSession4Test(tk.store) tk.require.NoError(err) se.SetConnectionID(asyncTestKitIDGenerator.Inc()) @@ -65,7 +65,7 @@ func (tk *AsyncTestKit) OpenSession(ctx context.Context, db string) context.Cont // CloseSession closes exists session from ctx. func (tk *AsyncTestKit) CloseSession(ctx context.Context) { - se := tryRetrieveSession(ctx) + se := TryRetrieveSession(ctx) tk.require.NotNil(se) se.Close() } @@ -135,7 +135,7 @@ func (tk *AsyncTestKit) ConcurrentRun( // Exec executes a sql statement. func (tk *AsyncTestKit) Exec(ctx context.Context, sql string, args ...interface{}) (sqlexec.RecordSet, error) { - se := tryRetrieveSession(ctx) + se := TryRetrieveSession(ctx) tk.require.NotNil(se) if len(args) == 0 { @@ -191,7 +191,7 @@ func (tk *AsyncTestKit) MustQuery(ctx context.Context, sql string, args ...inter // resultSetToResult converts ast.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. func (tk *AsyncTestKit) resultSetToResult(ctx context.Context, rs sqlexec.RecordSet, comment string) *Result { - rows, err := session.GetRows4Test(context.Background(), tryRetrieveSession(ctx), rs) + rows, err := session.GetRows4Test(context.Background(), TryRetrieveSession(ctx), rs) tk.require.NoError(err, comment) err = rs.Close() @@ -219,7 +219,8 @@ type sessionCtxKeyType struct{} var sessionKey = sessionCtxKeyType{} -func tryRetrieveSession(ctx context.Context) session.Session { +// TryRetrieveSession tries retrieve session from context. +func TryRetrieveSession(ctx context.Context) session.Session { s := ctx.Value(sessionKey) if s == nil { return nil diff --git a/testkit/handle.go b/testkit/handle.go index e911f164d8052..2c68608677fe2 100644 --- a/testkit/handle.go +++ b/testkit/handle.go @@ -30,8 +30,8 @@ import ( // MustNewCommonHandle create a common handle with given values. func MustNewCommonHandle(t *testing.T, values ...interface{}) kv.Handle { encoded, err := codec.EncodeKey(new(stmtctx.StatementContext), nil, types.MakeDatums(values...)...) - require.Nil(t, err) + require.NoError(t, err) ch, err := kv.NewCommonHandle(encoded) - require.Nil(t, err) + require.NoError(t, err) return ch } diff --git a/testkit/result.go b/testkit/result.go index 6950120d6a2a0..b9e288db09e44 100644 --- a/testkit/result.go +++ b/testkit/result.go @@ -99,3 +99,22 @@ func (res *Result) Rows() [][]interface{} { } return ifacesSlice } + +// CheckAt asserts the result of selected columns equals the expected results. +func (res *Result) CheckAt(cols []int, expected [][]interface{}) { + for _, e := range expected { + res.require.Equal(len(e), len(cols)) + } + + rows := make([][]string, 0, len(expected)) + for i := range res.rows { + row := make([]string, 0, len(cols)) + for _, r := range cols { + row = append(row, res.rows[i][r]) + } + rows = append(rows, row) + } + got := fmt.Sprintf("%s", rows) + need := fmt.Sprintf("%s", expected) + res.require.Equal(need, got, res.comment) +} diff --git a/testkit/testdata/testdata.go b/testkit/testdata/testdata.go index f9c9eae1e8c68..8ab2f343d7b8e 100644 --- a/testkit/testdata/testdata.go +++ b/testkit/testdata/testdata.go @@ -32,6 +32,7 @@ import ( "testing" "github.com/pingcap/errors" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/stretchr/testify/require" ) @@ -124,6 +125,14 @@ func ConvertRowsToStrings(rows [][]interface{}) (rs []string) { return rs } +// ConvertSQLWarnToStrings converts []SQLWarn to []string. +func ConvertSQLWarnToStrings(warns []stmtctx.SQLWarn) (rs []string) { + for _, warn := range warns { + rs = append(rs, fmt.Sprint(warn.Err.Error())) + } + return rs +} + // GetTestCases gets the test cases for a test function. func (td *TestData) GetTestCases(t *testing.T, in interface{}, out interface{}) { // Extract caller's name. @@ -151,6 +160,25 @@ func (td *TestData) GetTestCases(t *testing.T, in interface{}, out interface{}) td.output[casesIdx].decodedOut = out } +// GetTestCasesByName gets the test cases for a test function by its name. +func (td *TestData) GetTestCasesByName(caseName string, t *testing.T, in interface{}, out interface{}) { + casesIdx, ok := td.funcMap[caseName] + require.Truef(t, ok, "Case name: %s", caseName) + require.NoError(t, json.Unmarshal(*td.input[casesIdx].Cases, in)) + + if Record() { + inputLen := reflect.ValueOf(in).Elem().Len() + v := reflect.ValueOf(out).Elem() + if v.Kind() == reflect.Slice { + v.Set(reflect.MakeSlice(v.Type(), inputLen, inputLen)) + } + } else { + require.NoError(t, json.Unmarshal(*td.output[casesIdx].Cases, out)) + } + + td.output[casesIdx].decodedOut = out +} + func (td *TestData) generateOutputIfNeeded() error { if !record { return nil diff --git a/testkit/testkit.go b/testkit/testkit.go index c99791efe369a..2cd1f0111c70d 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -22,12 +22,18 @@ import ( "fmt" "strings" "testing" + "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/assert" @@ -57,6 +63,17 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit { } } +// NewTestKitWithSession returns a new *TestKit. +func NewTestKitWithSession(t testing.TB, store kv.Storage, se session.Session) *TestKit { + return &TestKit{ + require: require.New(t), + assert: assert.New(t), + t: t, + store: store, + session: se, + } +} + // RefreshSession set a new session for the testkit func (tk *TestKit) RefreshSession() { tk.session = newSession(tk.t, tk.store) @@ -93,6 +110,49 @@ func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { return tk.ResultSetToResult(rs, comment) } +// MustIndexLookup checks whether the plan for the sql is IndexLookUp. +func (tk *TestKit) MustIndexLookup(sql string, args ...interface{}) *Result { + tk.require.True(tk.HasPlan(sql, "IndexLookUp", args...)) + return tk.MustQuery(sql, args...) +} + +// MustPartition checks if the result execution plan must read specific partitions. +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { + rs := tk.MustQuery("explain "+sql, args...) + ok := len(partitions) == 0 + for i := range rs.rows { + if len(partitions) == 0 && strings.Contains(rs.rows[i][3], "partition:") { + ok = false + } + if len(partitions) != 0 && strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { + ok = true + } + } + tk.require.True(ok) + return tk.MustQuery(sql, args...) +} + +// MustPartitionByList checks if the result execution plan must read specific partitions by list. +func (tk *TestKit) MustPartitionByList(sql string, partitions []string, args ...interface{}) *Result { + rs := tk.MustQuery("explain "+sql, args...) + ok := len(partitions) == 0 + for i := range rs.rows { + if ok { + tk.require.NotContains(rs.rows[i][3], "partition:") + } + for index, partition := range partitions { + if !ok && strings.Contains(rs.rows[i][3], "partition:"+partition) { + partitions = append(partitions[:index], partitions[index+1:]...) + } + } + + } + if !ok { + tk.require.Len(partitions, 0) + } + return tk.MustQuery(sql, args...) +} + // QueryToErr executes a sql statement and discard results. func (tk *TestKit) QueryToErr(sql string, args ...interface{}) error { comment := fmt.Sprintf("sql:%s, args:%v", sql, args) @@ -258,9 +318,98 @@ func (tk *TestKit) CheckExecResult(affectedRows, insertID int64) { tk.require.Equal(int64(tk.Session().LastInsertID()), insertID) } +// MustPointGet checks whether the plan for the sql is Point_Get. +func (tk *TestKit) MustPointGet(sql string, args ...interface{}) *Result { + rs := tk.MustQuery("explain "+sql, args...) + tk.require.Len(rs.rows, 1) + tk.require.Contains(rs.rows[0][0], "Point_Get", "plan %v", rs.rows[0][0]) + return tk.MustQuery(sql, args...) +} + +// UsedPartitions returns the partition names that will be used or all/dual. +func (tk *TestKit) UsedPartitions(sql string, args ...interface{}) *Result { + rs := tk.MustQuery("explain "+sql, args...) + var usedPartitions [][]string + for i := range rs.rows { + index := strings.Index(rs.rows[i][3], "partition:") + if index != -1 { + p := rs.rows[i][3][index+len("partition:"):] + partitions := strings.Split(strings.SplitN(p, " ", 2)[0], ",") + usedPartitions = append(usedPartitions, partitions) + } + } + comment := fmt.Sprintf("sql:%s, args:%v", sql, args) + return &Result{rows: usedPartitions, comment: comment, assert: tk.assert, require: tk.require} +} + // WithPruneMode run test case under prune mode. func WithPruneMode(tk *TestKit, mode variable.PartitionPruneMode, f func()) { tk.MustExec("set @@tidb_partition_prune_mode=`" + string(mode) + "`") tk.MustExec("set global tidb_partition_prune_mode=`" + string(mode) + "`") f() } + +// MockGC is used to make GC work in the test environment. +func MockGC(tk *TestKit) (string, string, string, func()) { + originGC := ddl.IsEmulatorGCEnable() + resetGC := func() { + if originGC { + ddl.EmulatorGCEnable() + } else { + ddl.EmulatorGCDisable() + } + } + + // disable emulator GC. + // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. + ddl.EmulatorGCDisable() + gcTimeFormat := "20060102-15:04:05 -0700 MST" + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') + ON DUPLICATE KEY + UPDATE variable_value = '%[1]s'` + // clear GC variables first. + tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC +} + +func containGlobal(rs *Result) bool { + partitionNameCol := 2 + for i := range rs.rows { + if strings.Contains(rs.rows[i][partitionNameCol], "global") { + return true + } + } + return false +} + +// MustNoGlobalStats checks if there is no global stats. +func (tk *TestKit) MustNoGlobalStats(table string) bool { + if containGlobal(tk.MustQuery("show stats_meta where table_name like '" + table + "'")) { + return false + } + if containGlobal(tk.MustQuery("show stats_buckets where table_name like '" + table + "'")) { + return false + } + if containGlobal(tk.MustQuery("show stats_histograms where table_name like '" + table + "'")) { + return false + } + return true +} + +// TestGetTableByName gets table by name for test. +func TestGetTableByName(t *testing.T, ctx sessionctx.Context, db, table string) table.Table { + dom := domain.GetDomain(ctx) + // Make sure the table schema is the new schema. + err := dom.Reload() + require.NoError(t, err) + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table)) + require.NoError(t, err) + return tbl +} + +// CheckLastMessage checks last message after executing MustExec +func (tk *TestKit) CheckLastMessage(msg string) { + tk.require.Equal(tk.Session().LastMessage(), msg) +} diff --git a/tests/globalkilltest/global_kill_test.go b/tests/globalkilltest/global_kill_test.go index e0af71d35d70a..009bf6ed6c652 100644 --- a/tests/globalkilltest/global_kill_test.go +++ b/tests/globalkilltest/global_kill_test.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tidb/util/logutil" "github.com/stretchr/testify/require" - "go.etcd.io/etcd/clientv3" + clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" ) diff --git a/tests/readonlytest/readonly_test.go b/tests/readonlytest/readonly_test.go index bf76a8d91e593..bf2f267531788 100644 --- a/tests/readonlytest/readonly_test.go +++ b/tests/readonlytest/readonly_test.go @@ -28,8 +28,8 @@ import ( var ( tidbRootPassword = flag.String("passwd", "", "tidb root password") - tidbAPort = flag.Int("tidb_a_port", 4000, "first tidb server listening port") - tidbBPort = flag.Int("tidb_b_port", 4001, "second tidb server listening port") + tidbAPort = flag.Int("tidb_a_port", 4001, "first tidb server listening port") + tidbBPort = flag.Int("tidb_b_port", 4002, "second tidb server listening port") ReadOnlyErrMsg = "Error 1836: Running in read-only mode" ConflictErrMsg = "Error 1105: can't turn off tidb_super_read_only when tidb_restricted_read_only is on" PriviledgedErrMsg = "Error 1227: Access denied; you need (at least one of) the SUPER or SYSTEM_VARIABLES_ADMIN privilege(s) for this operation" @@ -107,8 +107,18 @@ func createReadOnlySuite(t *testing.T) (s *ReadOnlySuite, clean func()) { func TestRestriction(t *testing.T) { s, clean := createReadOnlySuite(t) defer clean() - setVariable(t, s.db, TiDBRestrictedReadOnly, 1) + var err error + _, err = s.db.Exec("drop table if exists t") + require.NoError(t, err) + _, err = s.udb.Exec("create table t (a int primary key, b int)") + require.NoError(t, err) + _, err = s.udb.Exec("insert into t values (1, 1)") + require.NoError(t, err) + _, err = s.udb.Exec("update t set b = 2 where a = 1") + require.NoError(t, err) + + setVariable(t, s.db, TiDBRestrictedReadOnly, 1) time.Sleep(1) checkVariable(t, s.udb, TiDBRestrictedReadOnly, true) @@ -117,7 +127,18 @@ func TestRestriction(t *testing.T) { checkVariable(t, s.rdb, TiDBRestrictedReadOnly, true) checkVariable(t, s.rdb, TiDBSuperReadOnly, true) - _, err := s.udb.Exec("create table t(a int)") + // can't create table + _, err = s.udb.Exec("create table t(a int)") + require.Error(t, err) + require.Equal(t, err.Error(), ReadOnlyErrMsg) + + // can't do point update when tidb_restricted_read_only is on + _, err = s.udb.Exec("update t set b = 2 where a = 1") + require.Error(t, err) + require.Equal(t, err.Error(), ReadOnlyErrMsg) + + // can't insert + _, err = s.udb.Exec("insert into t values (2, 3)") require.Error(t, err) require.Equal(t, err.Error(), ReadOnlyErrMsg) diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index 77c042daf8eae..99b7aec7b642d 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -30,7 +30,7 @@ var isCoverageServer string func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/tools/check/check-timeout.go b/tools/check/check-timeout.go deleted file mode 100644 index c377dcb1a2dc0..0000000000000 --- a/tools/check/check-timeout.go +++ /dev/null @@ -1,232 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package main - -import ( - "bufio" - "errors" - "fmt" - "os" - "strings" - "time" -) - -var allowList = make(map[string]struct{}) - -func init() { - tmp := []string{ - "TestT", - "TestCluster", - "testIntegrationSuite4.TestAddPartitionTooManyPartitions", - "testIntegrationSuite5.TestBackwardCompatibility", - "testIntegrationSuite1.TestDisableTablePartition", - "testPartitionSuite.TestAddPartitionReplicaBiggerThanTiFlashStores", - "testColumnTypeChangeSuite.TestColumnTypeChangeFromJsonToOthers", - "testColumnTypeChangeSuite.TestColumnTypeChangeFromNumericToOthers", - "testColumnTypeChangeSuite.TestColumnTypeChangeFromStringToOthers", - "testCTCSerialSuiteWrapper.TestColumnTypeChangeFromJsonToOthers", - "testCTCSerialSuiteWrapper.TestColumnTypeChangeFromNumericToOthers", - "testCTCSerialSuiteWrapper.TestColumnTypeChangeFromStringToOthers", - "testSerialDBSuite.TestCommitTxnWithIndexChange", - "testSerialDBSuite.TestDuplicateErrorMessage", - "TestAddIndexFailed", - "TestAddIndexWorkerNum", - "TestAddIndexWorkerNum/EnableClusteredIndex", - "TestAddIndexWorkerNum/DisableClusteredIndex", - "pkgTestSuite.TestAggPartialResultMapperB", - "testFastAnalyze.TestFastAnalyzeRetryRowCount", - "testSuite2.TestAddIndexPriority", - "testSuite3.TestAdminCheckPartitionTableFailed", - "testSuite.TestApplyColumnType", - "testSuiteJoin1.TestIndexLookupJoin", - "testSuiteJoin2.TestJoin", - "testSuite8.TestAdminCheckTable", - "testSuiteAgg.TestAggregation", - "testSuiteJoin3.TestSubquery", - "testSuiteJoin2.TestJoinCast", - "testSuite2.TestExplainAnalyzeCTEMemoryAndDiskInfo", - "testSuite7.TestSetWithCurrentTimestampAndNow", - "testSuite5.TestPartitionTableIndexJoinAndIndexReader", - "testSuite4.TestWriteListColumnsPartitionTable1", - "testSuite2.TestLowResolutionTSORead", - "testSuite1.TestPartitionTableRandomIndexMerge", - "testSuite3.TestPartitionTableIndexJoinIndexLookUp", - "testSuite4.TestWriteListPartitionTable2", - "partitionTableSuite.TestDML", - "partitionTableSuite.TestDirectReadingWithAgg", - "partitionTableSuite.TestDirectReadingWithUnionScan", - "partitionTableSuite.TestOrderByandLimit", - "partitionTableSuite.TestParallelApply", - "partitionTableSuite.TestSubqueries", - "partitionTableSuite.TestView", - "partitionTableSuite.TestUnsignedPartitionColumn", - "partitionTableSuite.TestGlobalStatsAndSQLBinding", - "partitionTableSuite.TestIdexMerge", - "partitionTableSuite.TestUnion", - "testRecoverTable.TestFlashbackTable", - "testRecoverTable.TestRecoverTable", - "tiflashTestSuite.TestCancelMppTasks", - "tiflashTestSuite.TestMppApply", - "tiflashTestSuite.TestMppExecution", - "tiflashTestSuite.TestMppUnionAll", - "tiflashTestSuite.TestPartitionTable", - "testSerialSuite.TestPrepareStmtAfterIsolationReadChange", - "testSerialSuite.TestSplitRegionTimeout", - "testSerialSuite.TestAggInDisk", - "testStaleTxnSerialSuite.TestSelectAsOf", - "testEvaluatorSuite.TestSleepVectorized", - "TestFailNewSession", - "testPlanSerialSuite.TestPartitionTable", - "testPlanSerialSuite.TestPartitionWithVariedDatasources", - "HTTPHandlerTestSuite.TestZipInfoForSQL", - "HTTPHandlerTestSuite.TestBinlogRecover", - "TestConnExecutionTimeout", - "TestTiFlashFallback", - "tidbTestTopSQLSuite.TestTopSQLCPUProfile", - "testPessimisticSuite.TestAmendForIndexChange", - "testPessimisticSuite.TestGenerateColPointGet", - "testPessimisticSuite.TestInnodbLockWaitTimeout", - "testPessimisticSuite.TestPessimisticLockNonExistsKey", - "testPessimisticSuite.TestSelectForUpdateNoWait", - "testSessionSerialSuite.TestProcessInfoIssue22068", - "TestRetryOpenStore", - "TestStoreErr", - "TestStoreSwitchPeer", - "testSequenceSuite.TestSequenceFunction", - "testSuiteP2.TestUnion", - "testVectorizeSuite1.TestVectorizedBuiltinTimeFuncGenerated", - "testSuiteJoin3.TestVectorizedShuffleMergeJoin", - "testSuite1.TestClusterIndexAnalyze", - "testSuiteJoin3.TestVectorizedMergeJoin", - "testFastAnalyze.TestAnalyzeFastSample", - "testBypassSuite.TestLatch", - "testIntegrationSuite2.TestPartitionCancelAddPrimaryKey", - "testDBSuite1.TestAddIndexWithSplitTable", - "testSerialDBSuite.TestAddIndexWithShardRowID", - "testColumnTypeChangeSuite.TestColumnTypeChangeFromDateTimeTypeToOthers", - "testSerialDBSuite1.TestAddPartitionReplicaBiggerThanTiFlashStores", - "TestMemStoreConcurrent", - } - for _, v := range tmp { - allowList[v] = struct{}{} - } -} - -func inAllowList(testName string) bool { - _, ok := allowList[testName] - return ok -} - -func parseLine(line string) (testName string, dur time.Duration, err error) { - // The line looks like that: - // PASS: sysvar_test.go:131: testSysVarSuite.TestIntValidation 0.000s - // PASS: sysvar_test.go:454: testSysVarSuite.TestIsNoop 0.000s - // PASS: sysvar_test.go:302: testSysVarSuite.TestMaxExecutionTime 0.000s - // PASS: sysvar_test.go:429: testSysVarSuite.TestReadOnlyNoop 0.000s - // --- PASS: TestSingle (0.26s) - // --- PASS: TestCluster (4.20s) - - line = strings.TrimSpace(line) - - // Format type 1 - if strings.HasPrefix(line, "PASS") { - return parseFormat1(line) - } - - // Format type 2 - if strings.HasPrefix(line, "---") { - return parseFormat2(line) - } - - err = errors.New("unknown format: " + line) - return -} - -func parseFormat1(line string) (testName string, dur time.Duration, err error) { - offset := strings.LastIndexByte(line, '\t') - if offset < 0 { - err = fmt.Errorf("get duration string error: %s", line) - return - } - durStr := line[offset+1:] - dur, err = time.ParseDuration(durStr) - if err != nil { - err = fmt.Errorf("parse duration string error: %s, %v", line, err) - return - } - - offset1 := strings.LastIndexByte(line[:offset], ' ') - if offset1 < 0 { - err = errors.New("parse line error: " + line) - return - } - testName = line[offset1+1 : offset] - return -} - -func parseFormat2(line string) (testName string, dur time.Duration, err error) { - offset := strings.LastIndexByte(line, ' ') - if offset < 0 { - err = fmt.Errorf("get duration string error: %s", line) - return - } - durStr := line[offset+2 : len(line)-1] - dur, err = time.ParseDuration(durStr) - if err != nil { - err = fmt.Errorf("parse duration string error: %s, %v", line, err) - return - } - - offset1 := strings.LastIndexByte(line[:offset], ' ') - if offset1 < 0 { - err = errors.New("parse line err: " + line) - return - } - testName = line[offset1+1 : offset] - return -} - -func main() { - lines := make([]string, 0, 100) - // They are generated by "grep 'PASS:' gotest.log", and gotest.log is generated by - // 'make gotest' from the Makefile which runs 'go test -v ./...' alike - scanner := bufio.NewScanner(os.Stdin) - for scanner.Scan() { - line := scanner.Text() - testName, dur, err := parseLine(line) - if err != nil { - fmt.Println("parser line error:", err) - os.Exit(-1) - } - if dur > 5*time.Second { - if inAllowList(testName) { - continue - } - lines = append(lines, line) - } - } - if err := scanner.Err(); err != nil { - fmt.Fprintln(os.Stderr, "reading standard input:", err) - os.Exit(-3) - } - - if len(lines) != 0 { - fmt.Println("The following test cases take too long to finish:") - for _, line := range lines { - fmt.Println(line) - } - os.Exit(-4) - } -} diff --git a/tools/check/go.mod b/tools/check/go.mod index 5db3382e2d8b9..dfefcbe6b57dc 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -15,6 +15,7 @@ require ( github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd go.uber.org/automaxprocs v1.4.0 // indirect + golang.org/x/tools v0.1.9 // indirect gopkg.in/alecthomas/gometalinter.v2 v2.0.12 // indirect gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index 53cc061d53eec..4ea5cdf60475b 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -90,6 +90,7 @@ github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UV github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/automaxprocs v1.4.0 h1:CpDZl6aOlLhReez+8S3eEotD7Jx0Os++lemPlMULQP0= @@ -101,15 +102,20 @@ golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.5.1 h1:OJxoQ/rynoF0dcCdI7cLPktw/hR2cueqYfjm43oqK38= +golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9 h1:SQFwaSi55rU7vdNs9Yr0Z324VNlrF+0wMqRXT4St8ck= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20171026204733-164713f0dfce/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -120,8 +126,11 @@ golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1 h1:SrN+KX8Art/Sf4HNj6Zcz06G7VEz+7w9tdXTPOZ7+l4= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211019181941-9d821ace8654 h1:id054HUawV2/6IGm2IV8KZQjqtwAOo2CYlOToYqa0d0= +golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1 h1:v+OssWQX+hTHEmOBgwxdZxK4zHq3yOs8F9J7mk0PY8E= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/text v0.0.0-20170915090833-1cbadb444a80/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -130,6 +139,9 @@ golang.org/x/text v0.3.2 h1:tW2bmiBqwgJj/UpqtC8EpXEZVYOwU0yG4iWbprSVAcs= golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= golang.org/x/text v0.3.3 h1:cokOdA+Jmi5PJGXLlLllQSgYigAEfHXJAERHVMaCc2k= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7 h1:olpwvP2KacW1ZWvsR7uQhoyTYvKAupfQrRGBFM352Gk= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/tools v0.0.0-20170915040203-e531a2a1c15f/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180911133044-677d2ff680c1/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -146,6 +158,8 @@ golang.org/x/tools v0.0.0-20200225230052-807dcd883420 h1:4RJNOV+2rLxMEfr6QIpC7GE golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= +golang.org/x/tools v0.1.9 h1:j9KsMiaP1c3B0OTQGth0/k+miLGTgLsAFUCrF2vLcF8= +golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= diff --git a/tools/check/ut.go b/tools/check/ut.go index 4e8c47a70bb5c..5e47cf13c1866 100644 --- a/tools/check/ut.go +++ b/tools/check/ut.go @@ -15,7 +15,9 @@ package main import ( + "bufio" "bytes" + "encoding/xml" "fmt" "io" "math/rand" @@ -23,12 +25,14 @@ import ( "os/exec" "path" "runtime" + "sort" "strings" "sync" "time" - // Set the correct when it runs inside docker. + // Set the correct value when it runs inside docker. _ "go.uber.org/automaxprocs" + "golang.org/x/tools/cover" ) func usage() bool { @@ -57,7 +61,11 @@ ut run $package $test ut build // build a test package -ut build xxx` +ut build xxx + +// write the junitfile +ut run --junitfile xxx` + fmt.Println(msg) return true } @@ -70,6 +78,10 @@ type task struct { old bool } +func (t *task) String() string { + return t.pkg + " " + t.test +} + var P int var workDir string @@ -133,12 +145,10 @@ func cmdBuild(args ...string) bool { // build all packages if len(args) == 0 { - for _, pkg := range pkgs { - err := buildTestBinary(pkg) - if err != nil { - fmt.Println("build package error", pkg, err) - return false - } + err := buildTestBinaryMulti(pkgs) + if err != nil { + fmt.Println("build package error", pkgs, err) + return false } return true } @@ -163,16 +173,16 @@ func cmdRun(args ...string) bool { return false } tasks := make([]task, 0, 5000) + start := time.Now() // run all tests if len(args) == 0 { - for _, pkg := range pkgs { - fmt.Println("handling package", pkg) - err := buildTestBinary(pkg) - if err != nil { - fmt.Println("build package error", pkg, err) - return false - } + err := buildTestBinaryMulti(pkgs) + if err != nil { + fmt.Println("build package error", pkgs, err) + return false + } + for _, pkg := range pkgs { exist, err := testBinaryExist(pkg) if err != nil { fmt.Println("check test binary existance error", err) @@ -248,33 +258,154 @@ func cmdRun(args ...string) bool { } tasks = tmp } - fmt.Println("building task finish...", len(tasks)) - numactl := numactlExist() + if except != "" { + list, err := parseCaseListFromFile(except) + if err != nil { + fmt.Println("parse --except file error", err) + return false + } + tmp := tasks[:0] + for _, task := range tasks { + if _, ok := list[task.String()]; !ok { + tmp = append(tmp, task) + } + } + tasks = tmp + } + + if only != "" { + list, err := parseCaseListFromFile(only) + if err != nil { + fmt.Println("parse --only file error", err) + return false + } + tmp := tasks[:0] + for _, task := range tasks { + if _, ok := list[task.String()]; ok { + tmp = append(tmp, task) + } + } + tasks = tmp + } + + fmt.Printf("building task finish, count=%d, takes=%v\n", len(tasks), time.Since(start)) + taskCh := make(chan task, 100) works := make([]numa, P) var wg sync.WaitGroup for i := 0; i < P; i++ { - works[i] = numa{fmt.Sprintf("%d", i), numactl, false} wg.Add(1) go works[i].worker(&wg, taskCh) } shuffle(tasks) + + start = time.Now() for _, task := range tasks { taskCh <- task } close(taskCh) wg.Wait() + fmt.Println("run all tasks takes", time.Since(start)) + + if junitfile != "" { + out := collectTestResults(works) + f, err := os.Create(junitfile) + if err != nil { + fmt.Println("create junit file fail:", err) + return false + } + if err := write(f, out); err != nil { + fmt.Println("write junit file error:", err) + return false + } + } + for _, work := range works { if work.Fail { return false } } + if coverprofile != "" { + collectCoverProfileFile() + } return true } +func parseCaseListFromFile(fileName string) (map[string]struct{}, error) { + f, err := os.Open(fileName) + if err != nil { + return nil, withTrace(err) + } + defer f.Close() + + ret := make(map[string]struct{}) + s := bufio.NewScanner(f) + for s.Scan() { + line := s.Bytes() + ret[string(line)] = struct{}{} + } + if err := s.Err(); err != nil { + return nil, withTrace(err) + } + return ret, nil +} + +// handleFlags strip the '--flag xxx' from the command line os.Args +// Example of the os.Args changes +// Before: ut run sessoin TestXXX --coverprofile xxx --junitfile yyy +// After: ut run session TestXXX +// The value of the flag is returned. +func handleFlags(flag string) string { + var res string + tmp := os.Args[:0] + // Iter to the flag + var i int + for ; i < len(os.Args); i++ { + if os.Args[i] == flag { + i++ + break + } + tmp = append(tmp, os.Args[i]) + } + // Handle the flag + if i < len(os.Args) { + res = os.Args[i] + i++ + } + // Iter the remain flags + for ; i < len(os.Args); i++ { + tmp = append(tmp, os.Args[i]) + } + + // os.Args is now the original flags with '--coverprofile XXX' removed. + os.Args = tmp + return res +} + +var junitfile string +var coverprofile string +var coverFileTempDir string + +var except string +var only string + func main() { + junitfile = handleFlags("--junitfile") + coverprofile = handleFlags("--coverprofile") + except = handleFlags("--except") + only = handleFlags("--only") + if coverprofile != "" { + var err error + coverFileTempDir, err = os.MkdirTemp(os.TempDir(), "cov") + if err != nil { + fmt.Println("create temp dir fail", coverFileTempDir) + os.Exit(1) + } + defer os.Remove(coverFileTempDir) + } + // Get the correct count of CPU if it's in docker. P = runtime.GOMAXPROCS(0) rand.Seed(time.Now().Unix()) @@ -284,14 +415,13 @@ func main() { fmt.Println("os.Getwd() error", err) } + var isSucceed bool if len(os.Args) == 1 { // run all tests - cmdRun() - return + isSucceed = cmdRun() } if len(os.Args) >= 2 { - var isSucceed bool switch os.Args[1] { case "list": isSucceed = cmdList(os.Args[2:]...) @@ -302,12 +432,161 @@ func main() { default: isSucceed = usage() } - if !isSucceed { - os.Exit(1) + } + if !isSucceed { + os.Exit(1) + } +} + +func collectCoverProfileFile() { + // Combine all the cover file of single test function into a whole. + files, err := os.ReadDir(coverFileTempDir) + if err != nil { + fmt.Println("collect cover file error:", err) + os.Exit(-1) + } + + w, err := os.Create(coverprofile) + if err != nil { + fmt.Println("create cover file error:", err) + os.Exit(-1) + } + defer w.Close() + w.WriteString("mode: set\n") + + result := make(map[string]*cover.Profile) + for _, file := range files { + if file.IsDir() { + continue + } + collectOneCoverProfileFile(result, file) + } + + w1 := bufio.NewWriter(w) + for _, prof := range result { + for _, block := range prof.Blocks { + fmt.Fprintf(w1, "%s:%d.%d,%d.%d %d %d\n", + prof.FileName, + block.StartLine, + block.StartCol, + block.EndLine, + block.EndCol, + block.NumStmt, + block.Count, + ) + } + if err := w1.Flush(); err != nil { + fmt.Println("flush data to cover profile file error:", err) + os.Exit(-1) } } } +func collectOneCoverProfileFile(result map[string]*cover.Profile, file os.DirEntry) { + f, err := os.Open(path.Join(coverFileTempDir, file.Name())) + if err != nil { + fmt.Println("open temp cover file error:", err) + os.Exit(-1) + } + defer f.Close() + + profs, err := cover.ParseProfilesFromReader(f) + if err != nil { + fmt.Println("parse cover profile file error:", err) + os.Exit(-1) + } + mergeProfile(result, profs) +} + +func mergeProfile(m map[string]*cover.Profile, profs []*cover.Profile) { + for _, prof := range profs { + sort.Sort(blocksByStart(prof.Blocks)) + old, ok := m[prof.FileName] + if !ok { + m[prof.FileName] = prof + continue + } + + // Merge samples from the same location. + // The data has already been sorted. + tmp := old.Blocks[:0] + var i, j int + for i < len(old.Blocks) && j < len(prof.Blocks) { + v1 := old.Blocks[i] + v2 := prof.Blocks[j] + + switch compareProfileBlock(v1, v2) { + case -1: + tmp = appendWithReduce(tmp, v1) + i++ + case 1: + tmp = appendWithReduce(tmp, v2) + j++ + default: + tmp = appendWithReduce(tmp, v1) + tmp = appendWithReduce(tmp, v2) + i++ + j++ + } + } + for ; i < len(old.Blocks); i++ { + tmp = appendWithReduce(tmp, old.Blocks[i]) + } + for ; j < len(prof.Blocks); j++ { + tmp = appendWithReduce(tmp, prof.Blocks[j]) + } + + m[prof.FileName] = old + } +} + +// appendWithReduce works like append(), but it merge the duplicated values. +func appendWithReduce(input []cover.ProfileBlock, b cover.ProfileBlock) []cover.ProfileBlock { + if len(input) >= 1 { + last := &input[len(input)-1] + if b.StartLine == last.StartLine && + b.StartCol == last.StartCol && + b.EndLine == last.EndLine && + b.EndCol == last.EndCol { + if b.NumStmt != last.NumStmt { + panic(fmt.Errorf("inconsistent NumStmt: changed from %d to %d", last.NumStmt, b.NumStmt)) + } + // Merge the data with the last one of the slice. + last.Count |= b.Count + return input + } + } + return append(input, b) +} + +type blocksByStart []cover.ProfileBlock + +func compareProfileBlock(x, y cover.ProfileBlock) int { + if x.StartLine < y.StartLine { + return -1 + } + if x.StartLine > y.StartLine { + return 1 + } + + // Now x.StartLine == y.StartLine + if x.StartCol < y.StartCol { + return -1 + } + if x.StartCol > y.StartCol { + return 1 + } + + return 0 +} + +func (b blocksByStart) Len() int { return len(b) } +func (b blocksByStart) Swap(i, j int) { b[i], b[j] = b[j], b[i] } +func (b blocksByStart) Less(i, j int) bool { + bi, bj := b[i], b[j] + return bi.StartLine < bj.StartLine || bi.StartLine == bj.StartLine && bi.StartCol < bj.StartCol +} + func listTestCases(pkg string, tasks []task) ([]task, error) { newCases, err := listNewTestCases(pkg) if err != nil { @@ -351,79 +630,119 @@ func listPackages() ([]string, error) { } type numa struct { - cpu string - numactl bool Fail bool + results []testResult } func (n *numa) worker(wg *sync.WaitGroup, ch chan task) { defer wg.Done() for t := range ch { - start := time.Now() res := n.runTestCase(t.pkg, t.test, t.old) - if res.err != nil { - fmt.Println("[FAIL] ", t.pkg, t.test, t.old, time.Since(start), res.err) - io.Copy(os.Stderr, &res.output) + if res.Failure != nil { + fmt.Println("[FAIL] ", t.pkg, t.test) + fmt.Fprintf(os.Stderr, "err=%s\n%s", res.err, res.Failure.Contents) n.Fail = true } + n.results = append(n.results, res) } } type testResult struct { - err error - output bytes.Buffer + JUnitTestCase + d time.Duration + err error } -func (n *numa) runTestCase(pkg string, fn string, old bool) (res testResult) { - exe := "./" + testFileName(pkg) - var cmd *exec.Cmd - if n.numactl { - cmd = n.testCommandWithNumaCtl(exe, fn, old) - } else { - cmd = n.testCommand(exe, fn, old) +func (n *numa) runTestCase(pkg string, fn string, old bool) testResult { + res := testResult{ + JUnitTestCase: JUnitTestCase{ + Classname: path.Join(modulePath, pkg), + Name: fn, + }, } + cmd := n.testCommand(pkg, fn, old) cmd.Dir = path.Join(workDir, pkg) // Combine the test case output, so the run result for failed cases can be displayed. - cmd.Stdout = &res.output - cmd.Stderr = &res.output + var buf bytes.Buffer + cmd.Stdout = &buf + cmd.Stderr = &buf + start := time.Now() if err := cmd.Run(); err != nil { - res.err = withTrace(err) + res.Failure = &JUnitFailure{ + Message: "Failed", + Contents: buf.String(), + } + res.err = err } + res.d = time.Since(start) + res.Time = formatDurationAsSeconds(res.d) return res } -func (n *numa) testCommandWithNumaCtl(exe string, fn string, old bool) *exec.Cmd { - if old { - // numactl --physcpubind 3 -- session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco - return exec.Command( - "numactl", "--physcpubind", n.cpu, "--", - exe, - "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) +func collectTestResults(workers []numa) JUnitTestSuites { + version := goVersion() + // pkg => test cases + pkgs := make(map[string][]JUnitTestCase) + durations := make(map[string]time.Duration) + + // The test result in workers are shuffled, so group by the packages here + for _, n := range workers { + for _, res := range n.results { + cases, ok := pkgs[res.Classname] + if !ok { + cases = make([]JUnitTestCase, 0, 10) + } + cases = append(cases, res.JUnitTestCase) + pkgs[res.Classname] = cases + durations[res.Classname] = durations[res.Classname] + res.d + } + } + + suites := JUnitTestSuites{} + // Turn every package result to a suite. + for pkg, cases := range pkgs { + suite := JUnitTestSuite{ + Tests: len(cases), + Failures: failureCases(cases), + Time: formatDurationAsSeconds(durations[pkg]), + Name: pkg, + Properties: packageProperties(version), + TestCases: cases, + } + suites.Suites = append(suites.Suites, suite) } + return suites +} - // numactl --physcpubind 3 -- session.test -test.run TestClusteredPrefixColum - return exec.Command( - "numactl", "--physcpubind", n.cpu, "--", - exe, - "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", fn) +func failureCases(input []JUnitTestCase) int { + sum := 0 + for _, v := range input { + if v.Failure != nil { + sum++ + } + } + return sum } -func (n *numa) testCommand(exe string, fn string, old bool) *exec.Cmd { +func (n *numa) testCommand(pkg string, fn string, old bool) *exec.Cmd { + args := make([]string, 0, 10) + exe := "./" + testFileName(pkg) + if coverprofile != "" { + fileName := strings.ReplaceAll(pkg, "/", "_") + "." + fn + tmpFile := path.Join(coverFileTempDir, fileName) + args = append(args, "-test.coverprofile", tmpFile) + } + args = append(args, "-test.cpu", "1") + args = append(args, []string{"-test.timeout", "2m"}...) if old { // session.test -test.run '^TestT$' -check.f testTxnStateSerialSuite.TestTxnInfoWithPSProtoco - return exec.Command( - exe, - "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", "^TestT$", "-check.f", fn) + args = append(args, "-test.run", "^TestT$", "-check.f", fn) + } else { + // session.test -test.run TestClusteredPrefixColum + args = append(args, "-test.run", fn) } - // session.test -test.run TestClusteredPrefixColum - return exec.Command( - exe, - "-test.timeout", "20s", - "-test.cpu", "1", "-test.run", fn) + return exec.Command(exe, args...) } func skipDIR(pkg string) bool { @@ -438,7 +757,12 @@ func skipDIR(pkg string) bool { func buildTestBinary(pkg string) error { // go test -c - cmd := exec.Command("go", "test", "-c", "-vet", "off", "-o", testFileName(pkg)) + var cmd *exec.Cmd + if coverprofile != "" { + cmd = exec.Command("go", "test", "-c", "-cover", "-vet", "off", "-o", testFileName(pkg)) + } else { + cmd = exec.Command("go", "test", "-c", "-vet", "off", "-o", testFileName(pkg)) + } cmd.Dir = path.Join(workDir, pkg) cmd.Stdout = os.Stdout cmd.Stderr = os.Stderr @@ -446,7 +770,31 @@ func buildTestBinary(pkg string) error { return withTrace(err) } return nil +} + +// buildTestBinaryMulti is much faster than build the test packages one by one. +func buildTestBinaryMulti(pkgs []string) error { + // go test --exec=xprog -cover -vet=off --count=0 $(pkgs) + xprogPath := path.Join(workDir, "tools/bin/xprog") + packages := make([]string, 0, len(pkgs)) + for _, pkg := range pkgs { + packages = append(packages, path.Join(modulePath, pkg)) + } + var cmd *exec.Cmd + if coverprofile != "" { + cmd = exec.Command("go", "test", "--exec", xprogPath, "-cover", "-vet", "off", "-count", "0") + } else { + cmd = exec.Command("go", "test", "--exec", xprogPath, "-vet", "off", "-count", "0") + } + cmd.Args = append(cmd.Args, packages...) + cmd.Dir = workDir + cmd.Stdout = os.Stdout + cmd.Stderr = os.Stderr + if err := cmd.Run(); err != nil { + return withTrace(err) + } + return nil } func testBinaryExist(pkg string) (bool, error) { @@ -458,13 +806,6 @@ func testBinaryExist(pkg string) (bool, error) { } return true, withTrace(err) } -func numactlExist() bool { - find, err := exec.Command("which", "numactl").Output() - if err == nil && len(find) > 0 { - return true - } - return false -} func testFileName(pkg string) string { _, file := path.Split(pkg) @@ -538,6 +879,7 @@ func filter(input []string, f func(string) bool) []string { } func shuffle(tasks []task) { + rand.Seed(time.Now().UnixNano()) for i := 0; i < len(tasks); i++ { pos := rand.Intn(len(tasks)) tasks[i], tasks[pos] = tasks[pos], tasks[i] @@ -564,3 +906,90 @@ func withTrace(err error) error { sz := runtime.Stack(stack[:], false) return &errWithStack{err, stack[:sz]} } + +func formatDurationAsSeconds(d time.Duration) string { + return fmt.Sprintf("%f", d.Seconds()) +} + +func packageProperties(goVersion string) []JUnitProperty { + return []JUnitProperty{ + {Name: "go.version", Value: goVersion}, + } +} + +// goVersion returns the version as reported by the go binary in PATH. This +// version will not be the same as runtime.Version, which is always the version +// of go used to build the gotestsum binary. +// +// To skip the os/exec call set the GOVERSION environment variable to the +// desired value. +func goVersion() string { + if version, ok := os.LookupEnv("GOVERSION"); ok { + return version + } + cmd := exec.Command("go", "version") + out, err := cmd.Output() + if err != nil { + return "unknown" + } + return strings.TrimPrefix(strings.TrimSpace(string(out)), "go version ") +} + +func write(out io.Writer, suites JUnitTestSuites) error { + doc, err := xml.MarshalIndent(suites, "", "\t") + if err != nil { + return err + } + _, err = out.Write([]byte(xml.Header)) + if err != nil { + return err + } + _, err = out.Write(doc) + return err +} + +// JUnitTestSuites is a collection of JUnit test suites. +type JUnitTestSuites struct { + XMLName xml.Name `xml:"testsuites"` + Suites []JUnitTestSuite +} + +// JUnitTestSuite is a single JUnit test suite which may contain many +// testcases. +type JUnitTestSuite struct { + XMLName xml.Name `xml:"testsuite"` + Tests int `xml:"tests,attr"` + Failures int `xml:"failures,attr"` + Time string `xml:"time,attr"` + Name string `xml:"name,attr"` + Properties []JUnitProperty `xml:"properties>property,omitempty"` + TestCases []JUnitTestCase +} + +// JUnitTestCase is a single test case with its result. +type JUnitTestCase struct { + XMLName xml.Name `xml:"testcase"` + Classname string `xml:"classname,attr"` + Name string `xml:"name,attr"` + Time string `xml:"time,attr"` + SkipMessage *JUnitSkipMessage `xml:"skipped,omitempty"` + Failure *JUnitFailure `xml:"failure,omitempty"` +} + +// JUnitSkipMessage contains the reason why a testcase was skipped. +type JUnitSkipMessage struct { + Message string `xml:"message,attr"` +} + +// JUnitProperty represents a key/value pair used to define properties. +type JUnitProperty struct { + Name string `xml:"name,attr"` + Value string `xml:"value,attr"` +} + +// JUnitFailure contains data related to a failed test. +type JUnitFailure struct { + Message string `xml:"message,attr"` + Type string `xml:"type,attr"` + Contents string `xml:",chardata"` +} diff --git a/tools/check/xprog.go b/tools/check/xprog.go new file mode 100644 index 0000000000000..f6b1c6357b012 --- /dev/null +++ b/tools/check/xprog.go @@ -0,0 +1,118 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "bufio" + "fmt" + "io" + "os" + "path/filepath" + "strings" +) + +func main() { + // See https://github.com/golang/go/issues/15513#issuecomment-773994959 + // go test --exec=xprog ./... + // Command line args looks like: + // '$CWD/xprog /tmp/go-build2662369829/b1382/aggfuncs.test -test.paniconexit0 -test.timeout=10m0s' + // This program moves the test binary /tmp/go-build2662369829/b1382/aggfuncs.test to someplace else for later use. + + // Extract the current work directory + cwd := os.Args[0] + cwd = cwd[:len(cwd)-len("tools/bin/xprog")] + + testBinaryPath := os.Args[1] + dir, _ := filepath.Split(testBinaryPath) + + // Extract the package info from /tmp/go-build2662369829/b1382/importcfg.link + pkg := getPackageInfo(dir) + + const prefix = "github.com/pingcap/tidb/" + if !strings.HasPrefix(pkg, prefix) { + os.Exit(-3) + } + + // github.com/pingcap/tidb/util/topsql.test => util/topsql + pkg = pkg[len(prefix) : len(pkg)-len(".test")] + + _, file := filepath.Split(pkg) + + // The path of the destination file looks like $CWD/util/topsql/topsql.test.bin + newName := filepath.Join(cwd, pkg, file+".test.bin") + + if err1 := os.Rename(testBinaryPath, newName); err1 != nil { + // Rename fail, handle error like "invalid cross-device linkcd tools/check" + err1 = MoveFile(testBinaryPath, newName) + if err1 != nil { + os.Exit(-4) + } + } +} + +func getPackageInfo(dir string) string { + // Read the /tmp/go-build2662369829/b1382/importcfg.link file to get the package information + f, err := os.Open(filepath.Join(dir, "importcfg.link")) + if err != nil { + os.Exit(-1) + } + defer f.Close() + + r := bufio.NewReader(f) + // packagefile github.com/pingcap/tidb/session.test=/home/genius/.cache/go-build/fb/fb1587cce5727fa9461131eab8260a52878da04f5c8da49dd3c7b2d941430c63-d + line, _, err := r.ReadLine() + if err != nil { + os.Exit(-2) + } + start := strings.IndexByte(string(line), ' ') + end := strings.IndexByte(string(line), '=') + pkg := string(line[start+1 : end]) + return pkg +} + +func MoveFile(sourcePath, destPath string) error { + inputFile, err := os.Open(sourcePath) + if err != nil { + return fmt.Errorf("Couldn't open source file: %s", err) + } + outputFile, err := os.Create(destPath) + if err != nil { + inputFile.Close() + return fmt.Errorf("Couldn't open dest file: %s", err) + } + defer outputFile.Close() + _, err = io.Copy(outputFile, inputFile) + inputFile.Close() + if err != nil { + return fmt.Errorf("Writing to output file failed: %s", err) + } + + // Handle the permissions + si, err := os.Stat(sourcePath) + if err != nil { + return fmt.Errorf("Stat error: %s", err) + } + err = os.Chmod(destPath, si.Mode()) + if err != nil { + return fmt.Errorf("Chmod error: %s", err) + } + + // The copy was successful, so now delete the original file + err = os.Remove(sourcePath) + if err != nil { + return fmt.Errorf("Failed removing original file: %s", err) + } + return nil +} diff --git a/types/convert.go b/types/convert.go index 95692e85b7675..3fdfdf810389a 100644 --- a/types/convert.go +++ b/types/convert.go @@ -301,7 +301,7 @@ func StrToUint(sc *stmtctx.StatementContext, str string, isFuncCast bool) (uint6 } // StrToDateTime converts str to MySQL DateTime. -func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int8) (Time, error) { +func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int) (Time, error) { return ParseTime(sc, str, mysql.TypeDatetime, fsp) } @@ -309,7 +309,7 @@ func StrToDateTime(sc *stmtctx.StatementContext, str string, fsp int8) (Time, er // and returns Time when str is in datetime format. // when isDuration is true, the d is returned, when it is false, the t is returned. // See https://dev.mysql.com/doc/refman/5.5/en/date-and-time-literals.html. -func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int8) (d Duration, t Time, isDuration bool, err error) { +func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int) (d Duration, t Time, isDuration bool, err error) { str = strings.TrimSpace(str) length := len(str) if length > 0 && str[0] == '-' { @@ -332,7 +332,7 @@ func StrToDuration(sc *stmtctx.StatementContext, str string, fsp int8) (d Durati } // NumberToDuration converts number to Duration. -func NumberToDuration(number int64, fsp int8) (Duration, error) { +func NumberToDuration(number int64, fsp int) (Duration, error) { if number > TimeMaxValue { // Try to parse DATETIME. if number >= 10000000000 { // '2001-00-00 00-00-00' diff --git a/types/convert_test.go b/types/convert_test.go index 94b60edb76617..7df7fff64f487 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -1175,7 +1175,7 @@ func TestNumberToDuration(t *testing.T) { } for _, tc := range testCases { - dur, err := NumberToDuration(tc.number, int8(tc.fsp)) + dur, err := NumberToDuration(tc.number, tc.fsp) if tc.hasErr { require.Error(t, err) continue @@ -1205,7 +1205,7 @@ func TestStrToDuration(t *testing.T) { sc := new(stmtctx.StatementContext) var tests = []struct { str string - fsp int8 + fsp int isDuration bool }{ {"20190412120000", 4, false}, diff --git a/types/datum.go b/types/datum.go index 5d48a3c326b08..ba1c58243993f 100644 --- a/types/datum.go +++ b/types/datum.go @@ -345,7 +345,7 @@ func (d *Datum) SetMysqlDecimal(b *MyDecimal) { // GetMysqlDuration gets Duration value func (d *Datum) GetMysqlDuration() Duration { - return Duration{Duration: time.Duration(d.i), Fsp: int8(d.decimal)} + return Duration{Duration: time.Duration(d.i), Fsp: int(int8(d.decimal))} } // SetMysqlDuration sets Duration value @@ -1136,10 +1136,9 @@ func (d *Datum) convertToUint(sc *stmtctx.StatementContext, target *FieldType) ( return ret, errors.Trace(err1) } val, err = ConvertUintToUint(uval, upperBound, tp) - if err != nil { - return ret, errors.Trace(err) + if err == nil { + err = err1 } - err = err1 case KindMysqlTime: dec := d.GetMysqlTime().ToNumber() err = dec.Round(dec, 0, ModeHalfEven) @@ -1191,7 +1190,7 @@ func (d *Datum) convertToMysqlTimestamp(sc *stmtctx.StatementContext, target *Fi ) fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = int8(target.Decimal) + fsp = target.Decimal } switch d.k { case KindMysqlTime: @@ -1239,7 +1238,7 @@ func (d *Datum) convertToMysqlTime(sc *stmtctx.StatementContext, target *FieldTy tp := target.Tp fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = int8(target.Decimal) + fsp = target.Decimal } var ( ret Datum @@ -1302,7 +1301,7 @@ func (d *Datum) convertToMysqlDuration(sc *stmtctx.StatementContext, target *Fie tp := target.Tp fsp := DefaultFsp if target.Decimal != UnspecifiedLength { - fsp = int8(target.Decimal) + fsp = target.Decimal } var ret Datum switch d.k { diff --git a/types/datum_test.go b/types/datum_test.go index e1f2f7aaa1bb4..063f1703f9537 100644 --- a/types/datum_test.go +++ b/types/datum_test.go @@ -149,6 +149,38 @@ func TestToInt64(t *testing.T) { testDatumToInt64(t, v, int64(3)) } +func testDatumToUInt32(t *testing.T, val interface{}, expect uint32, hasError bool) { + d := NewDatum(val) + sc := new(stmtctx.StatementContext) + sc.IgnoreTruncate = true + + ft := NewFieldType(mysql.TypeLong) + ft.Flag |= mysql.UnsignedFlag + converted, err := d.ConvertTo(sc, ft) + + if hasError { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + require.Equal(t, KindUint64, converted.Kind()) + require.Equal(t, uint64(expect), converted.GetUint64()) +} + +func TestToUint32(t *testing.T) { + // test overflow + testDatumToUInt32(t, 5000000000, 4294967295, true) + testDatumToUInt32(t, int64(-1), 4294967295, true) + testDatumToUInt32(t, "5000000000", 4294967295, true) + + testDatumToUInt32(t, 12345, 12345, false) + testDatumToUInt32(t, int64(0), 0, false) + testDatumToUInt32(t, 2147483648, 2147483648, false) + testDatumToUInt32(t, Enum{Name: "a", Value: 1}, 1, false) + testDatumToUInt32(t, Set{Name: "a", Value: 1}, 1, false) +} + func TestConvertToFloat(t *testing.T) { testCases := []struct { d Datum @@ -192,7 +224,7 @@ func TestConvertToFloat(t *testing.T) { } // mustParseTimeIntoDatum is similar to ParseTime but panic if any error occurs. -func mustParseTimeIntoDatum(s string, tp byte, fsp int8) (d Datum) { +func mustParseTimeIntoDatum(s string, tp byte, fsp int) (d Datum) { t, err := ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, s, tp, fsp) if err != nil { panic("ParseTime fail") diff --git a/types/enum_test.go b/types/enum_test.go index 8c23ba2d4e707..c7a349a10245d 100644 --- a/types/enum_test.go +++ b/types/enum_test.go @@ -18,14 +18,10 @@ import ( "testing" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) func TestEnum(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - t.Run("ParseEnum", func(t *testing.T) { tests := []struct { Elems []string diff --git a/types/field_type.go b/types/field_type.go index aadd3e5d19a67..0dd3343a48643 100644 --- a/types/field_type.go +++ b/types/field_type.go @@ -286,18 +286,18 @@ func DefaultTypeForValue(value interface{}, tp *FieldType, char string, collate case mysql.TypeDatetime, mysql.TypeTimestamp: tp.Flen = mysql.MaxDatetimeWidthNoFsp if x.Fsp() > DefaultFsp { // consider point('.') and the fractional part. - tp.Flen += int(x.Fsp()) + 1 + tp.Flen += x.Fsp() + 1 } - tp.Decimal = int(x.Fsp()) + tp.Decimal = x.Fsp() } SetBinChsClnFlag(tp) case Duration: tp.Tp = mysql.TypeDuration tp.Flen = len(x.String()) if x.Fsp > DefaultFsp { // consider point('.') and the fractional part. - tp.Flen = int(x.Fsp) + 1 + tp.Flen = x.Fsp + 1 } - tp.Decimal = int(x.Fsp) + tp.Decimal = x.Fsp SetBinChsClnFlag(tp) case *MyDecimal: tp.Tp = mysql.TypeNewDecimal diff --git a/types/fsp.go b/types/fsp.go index ffad73c42a48c..65962938cacc7 100644 --- a/types/fsp.go +++ b/types/fsp.go @@ -24,48 +24,48 @@ import ( const ( // UnspecifiedFsp is the unspecified fractional seconds part. - UnspecifiedFsp = int8(-1) + UnspecifiedFsp = -1 // MaxFsp is the maximum digit of fractional seconds part. - MaxFsp = int8(6) + MaxFsp = 6 // MinFsp is the minimum digit of fractional seconds part. - MinFsp = int8(0) + MinFsp = 0 // DefaultFsp is the default digit of fractional seconds part. // MySQL use 0 as the default Fsp. - DefaultFsp = int8(0) + DefaultFsp = 0 ) // CheckFsp checks whether fsp is in valid range. -func CheckFsp(fsp int) (int8, error) { - if fsp == int(UnspecifiedFsp) { +func CheckFsp(fsp int) (int, error) { + if fsp == UnspecifiedFsp { return DefaultFsp, nil } - if fsp < int(MinFsp) { + if fsp < MinFsp { return DefaultFsp, errors.Errorf("Invalid fsp %d", fsp) - } else if fsp > int(MaxFsp) { + } else if fsp > MaxFsp { return MaxFsp, nil } - return int8(fsp), nil + return fsp, nil } // ParseFrac parses the input string according to fsp, returns the microsecond, // and also a bool value to indice overflow. eg: // "999" fsp=2 will overflow. -func ParseFrac(s string, fsp int8) (v int, overflow bool, err error) { +func ParseFrac(s string, fsp int) (v int, overflow bool, err error) { if len(s) == 0 { return 0, false, nil } - fsp, err = CheckFsp(int(fsp)) + fsp, err = CheckFsp(fsp) if err != nil { return 0, false, errors.Trace(err) } - if int(fsp) >= len(s) { + if fsp >= len(s) { tmp, e := strconv.ParseInt(s, 10, 64) if e != nil { return 0, false, errors.Trace(e) } - v = int(float64(tmp) * math.Pow10(int(MaxFsp)-len(s))) + v = int(float64(tmp) * math.Pow10(MaxFsp-len(s))) return } @@ -76,7 +76,7 @@ func ParseFrac(s string, fsp int8) (v int, overflow bool, err error) { } tmp = (tmp + 5) / 10 - if float64(tmp) >= math.Pow10(int(fsp)) { + if float64(tmp) >= math.Pow10(fsp) { // overflow return 0, true, nil } @@ -85,7 +85,7 @@ func ParseFrac(s string, fsp int8) (v int, overflow bool, err error) { // 1236 round 3 -> 124 -> 124000 // 0312 round 2 -> 3 -> 30000 // 999 round 2 -> 100 -> overflow - v = int(float64(tmp) * math.Pow10(int(MaxFsp-fsp))) + v = int(float64(tmp) * math.Pow10(MaxFsp-fsp)) return } diff --git a/types/fsp_test.go b/types/fsp_test.go index 1daa31ee82c72..05d180dfed446 100644 --- a/types/fsp_test.go +++ b/types/fsp_test.go @@ -22,7 +22,7 @@ import ( ) func TestCheckFsp(t *testing.T) { - obtained, err := CheckFsp(int(UnspecifiedFsp)) + obtained, err := CheckFsp(UnspecifiedFsp) require.Equal(t, DefaultFsp, obtained) require.NoError(t, err) @@ -30,33 +30,33 @@ func TestCheckFsp(t *testing.T) { require.Equal(t, DefaultFsp, obtained) require.EqualError(t, err, "Invalid fsp -2019") - obtained, err = CheckFsp(int(MinFsp) - 4294967296) + obtained, err = CheckFsp(MinFsp - 4294967296) require.Equal(t, DefaultFsp, obtained) - require.EqualError(t, err, "Invalid fsp "+strconv.Itoa(int(MinFsp)-4294967296)) + require.EqualError(t, err, "Invalid fsp "+strconv.Itoa(MinFsp-4294967296)) // UnspecifiedFsp obtained, err = CheckFsp(-1) require.Equal(t, DefaultFsp, obtained) require.NoError(t, err) - obtained, err = CheckFsp(int(MaxFsp) + 1) + obtained, err = CheckFsp(MaxFsp + 1) require.Equal(t, MaxFsp, obtained) require.NoError(t, err) - obtained, err = CheckFsp(int(MaxFsp) + 2019) + obtained, err = CheckFsp(MaxFsp + 2019) require.Equal(t, MaxFsp, obtained) require.NoError(t, err) - obtained, err = CheckFsp(int(MaxFsp) + 4294967296) + obtained, err = CheckFsp(MaxFsp + 4294967296) require.Equal(t, MaxFsp, obtained) require.NoError(t, err) - obtained, err = CheckFsp(int(MaxFsp+MinFsp) / 2) + obtained, err = CheckFsp((MaxFsp + MinFsp) / 2) require.Equal(t, (MaxFsp+MinFsp)/2, obtained) require.NoError(t, err) obtained, err = CheckFsp(5) - require.Equal(t, int8(5), obtained) + require.Equal(t, 5, obtained) require.NoError(t, err) } @@ -67,7 +67,7 @@ func TestParseFrac(t *testing.T) { require.NoError(t, err) a := 200 - obtained, overflow, err = ParseFrac("999", int8(a)) + obtained, overflow, err = ParseFrac("999", int(int8(a))) require.Equal(t, 0, obtained) require.False(t, overflow) require.Error(t, err) diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index d7c9763d09c4c..88770ae78c89f 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -988,8 +988,8 @@ func ContainsBinary(obj, target BinaryJSON) bool { switch obj.TypeCode { case TypeCodeObject: if target.TypeCode == TypeCodeObject { - len := target.GetElemCount() - for i := 0; i < len; i++ { + elemCount := target.GetElemCount() + for i := 0; i < elemCount; i++ { key := target.objectGetKey(i) val := target.objectGetVal(i) if exp, exists := obj.objectSearchKey(key); !exists || !ContainsBinary(exp, val) { @@ -1001,16 +1001,16 @@ func ContainsBinary(obj, target BinaryJSON) bool { return false case TypeCodeArray: if target.TypeCode == TypeCodeArray { - len := target.GetElemCount() - for i := 0; i < len; i++ { + elemCount := target.GetElemCount() + for i := 0; i < elemCount; i++ { if !ContainsBinary(obj, target.arrayGetElem(i)) { return false } } return true } - len := obj.GetElemCount() - for i := 0; i < len; i++ { + elemCount := obj.GetElemCount() + for i := 0; i < elemCount; i++ { if ContainsBinary(obj.arrayGetElem(i), target) { return true } @@ -1034,9 +1034,9 @@ func ContainsBinary(obj, target BinaryJSON) bool { func (bj BinaryJSON) GetElemDepth() int { switch bj.TypeCode { case TypeCodeObject: - len := bj.GetElemCount() + elemCount := bj.GetElemCount() maxDepth := 0 - for i := 0; i < len; i++ { + for i := 0; i < elemCount; i++ { obj := bj.objectGetVal(i) depth := obj.GetElemDepth() if depth > maxDepth { @@ -1045,9 +1045,9 @@ func (bj BinaryJSON) GetElemDepth() int { } return maxDepth + 1 case TypeCodeArray: - len := bj.GetElemCount() + elemCount := bj.GetElemCount() maxDepth := 0 - for i := 0; i < len; i++ { + for i := 0; i < elemCount; i++ { obj := bj.arrayGetElem(i) depth := obj.GetElemDepth() if depth > maxDepth { diff --git a/types/main_test.go b/types/main_test.go index 73d0d58061bda..d502408d4926e 100644 --- a/types/main_test.go +++ b/types/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/types/set_test.go b/types/set_test.go index adfe78767f8f5..fcc4fe5b850ee 100644 --- a/types/set_test.go +++ b/types/set_test.go @@ -18,14 +18,10 @@ import ( "testing" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) func TestSet(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) - elems := []string{"a", "b", "c", "d"} t.Run("ParseSet", func(t *testing.T) { diff --git a/types/time.go b/types/time.go index 3d39b76eaf6bb..33cf632f1c832 100644 --- a/types/time.go +++ b/types/time.go @@ -255,16 +255,16 @@ const ( microsecondBitFieldMask uint64 = ((1 << microsecondBitFieldWidth) - 1) << microsecondBitFieldOffset fspTtBitFieldMask uint64 = ((1 << fspTtBitFieldWidth) - 1) << fspTtBitFieldOffset - fspTtForDate uint8 = 0b1110 + fspTtForDate uint = 0b1110 fspBitFieldMask uint64 = 0b1110 coreTimeBitFieldMask = ^fspTtBitFieldMask ) // NewTime constructs time from core time, type and fsp. -func NewTime(coreTime CoreTime, tp uint8, fsp int8) Time { +func NewTime(coreTime CoreTime, tp uint8, fsp int) Time { t := ZeroTime p := (*uint64)(&t.coreTime) - *p |= (uint64(coreTime) & coreTimeBitFieldMask) + *p |= uint64(coreTime) & coreTimeBitFieldMask if tp == mysql.TypeDate { *p |= uint64(fspTtForDate) return t @@ -279,11 +279,11 @@ func NewTime(coreTime CoreTime, tp uint8, fsp int8) Time { return t } -func (t Time) getFspTt() uint8 { - return uint8(uint64(t.coreTime) & fspTtBitFieldMask) +func (t Time) getFspTt() uint { + return uint(uint64(t.coreTime) & fspTtBitFieldMask) } -func (t *Time) setFspTt(fspTt uint8) { +func (t *Time) setFspTt(fspTt uint) { *(*uint64)(&t.coreTime) &= ^(fspTtBitFieldMask) *(*uint64)(&t.coreTime) |= uint64(fspTt) } @@ -300,12 +300,12 @@ func (t Time) Type() uint8 { } // Fsp returns fsp value. -func (t Time) Fsp() int8 { +func (t Time) Fsp() int { fspTt := t.getFspTt() if fspTt == fspTtForDate { return 0 } - return int8(fspTt >> 1) + return int(fspTt >> 1) } // SetType updates the type in Time. @@ -321,13 +321,13 @@ func (t *Time) SetType(tp uint8) { case mysql.TypeTimestamp: fspTt |= 1 case mysql.TypeDatetime: - fspTt &= ^(uint8(1)) + fspTt &= ^(uint(1)) } t.setFspTt(fspTt) } // SetFsp updates the fsp in Time. -func (t *Time) SetFsp(fsp int8) { +func (t *Time) SetFsp(fsp int) { if t.getFspTt() == fspTtForDate { return } @@ -335,7 +335,7 @@ func (t *Time) SetFsp(fsp int8) { fsp = DefaultFsp } *(*uint64)(&t.coreTime) &= ^(fspBitFieldMask) - *(*uint64)(&t.coreTime) |= (uint64(fsp) << 1) + *(*uint64)(&t.coreTime) |= uint64(fsp) << 1 } // CoreTime returns core time. @@ -346,7 +346,7 @@ func (t Time) CoreTime() CoreTime { // SetCoreTime updates core time. func (t *Time) SetCoreTime(ct CoreTime) { *(*uint64)(&t.coreTime) &= ^coreTimeBitFieldMask - *(*uint64)(&t.coreTime) |= (uint64(ct) & coreTimeBitFieldMask) + *(*uint64)(&t.coreTime) |= uint64(ct) & coreTimeBitFieldMask } // CurrentTime returns current time with type tp. @@ -436,7 +436,7 @@ func (t Time) FillNumber(dec *MyDecimal) { fsp := t.Fsp() if fsp > 0 { s1 := fmt.Sprintf("%s.%06d", s, t.Microsecond()) - s = s1[:len(s)+int(fsp)+1] + s = s1[:len(s)+fsp+1] } // We skip checking error here because time formatted string can be parsed certainly. err = dec.FromString([]byte(s)) @@ -492,19 +492,19 @@ func (t Time) CompareString(sc *stmtctx.StatementContext, str string) (int, erro } // roundTime rounds the time value according to digits count specified by fsp. -func roundTime(t gotime.Time, fsp int8) gotime.Time { - d := gotime.Duration(math.Pow10(9 - int(fsp))) +func roundTime(t gotime.Time, fsp int) gotime.Time { + d := gotime.Duration(math.Pow10(9 - fsp)) return t.Round(d) } // RoundFrac rounds the fraction part of a time-type value according to `fsp`. -func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int8) (Time, error) { +func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int) (Time, error) { if t.Type() == mysql.TypeDate || t.IsZero() { // date type has no fsp return t, nil } - fsp, err := CheckFsp(int(fsp)) + fsp, err := CheckFsp(fsp) if err != nil { return t, errors.Trace(err) } @@ -542,7 +542,7 @@ func (t Time) RoundFrac(sc *stmtctx.StatementContext, fsp int8) (Time, error) { } // GetFsp gets the fsp of a string. -func GetFsp(s string) int8 { +func GetFsp(s string) int { index := GetFracIndex(s) var fsp int if index < 0 { @@ -553,7 +553,7 @@ func GetFsp(s string) int8 { if fsp > 6 { fsp = 6 } - return int8(fsp) + return fsp } // GetFracIndex finds the last '.' for get fracStr, index = -1 means fracStr not found. @@ -576,22 +576,22 @@ func GetFracIndex(s string) (index int) { // We will use the “round half up” rule, e.g, >= 0.5 -> 1, < 0.5 -> 0, // so 2011:11:11 10:10:10.888888 round 0 -> 2011:11:11 10:10:11 // and 2011:11:11 10:10:10.111111 round 0 -> 2011:11:11 10:10:10 -func RoundFrac(t gotime.Time, fsp int8) (gotime.Time, error) { - _, err := CheckFsp(int(fsp)) +func RoundFrac(t gotime.Time, fsp int) (gotime.Time, error) { + _, err := CheckFsp(fsp) if err != nil { return t, errors.Trace(err) } - return t.Round(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond), nil //nolint:durationcheck + return t.Round(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond), nil //nolint:durationcheck } // TruncateFrac truncates fractional seconds precision with new fsp and returns a new one. // 2011:11:11 10:10:10.888888 round 0 -> 2011:11:11 10:10:10 // 2011:11:11 10:10:10.111111 round 0 -> 2011:11:11 10:10:10 -func TruncateFrac(t gotime.Time, fsp int8) (gotime.Time, error) { - if _, err := CheckFsp(int(fsp)); err != nil { +func TruncateFrac(t gotime.Time, fsp int) (gotime.Time, error) { + if _, err := CheckFsp(fsp); err != nil { return t, err } - return t.Truncate(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond), nil //nolint:durationcheck + return t.Truncate(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond), nil //nolint:durationcheck } // ToPackedUint encodes Time to a packed uint64 value. @@ -786,7 +786,8 @@ func isValidSeparator(c byte, prevParts int) bool { return true } - if prevParts == 2 && (c == ' ' || c == 'T') { + // for https://github.com/pingcap/tidb/issues/32232 + if prevParts == 2 && (c == 'T' || c == ' ' || c == '\t' || c == '\n' || c == '\v' || c == '\f' || c == '\r') { return true } @@ -923,7 +924,7 @@ func splitDateTime(format string) (seps []string, fracStr string, hasTZ bool, tz } // See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-literals.html. -func parseDatetime(sc *stmtctx.StatementContext, str string, fsp int8, isFloat bool) (Time, error) { +func parseDatetime(sc *stmtctx.StatementContext, str string, fsp int, isFloat bool) (Time, error) { var ( year, month, day, hour, minute, second, deltaHour, deltaMinute int fracStr string @@ -1270,7 +1271,7 @@ func AdjustYear(y int64, adjustZero bool) (int64, error) { } // NewDuration construct duration with time. -func NewDuration(hour, minute, second, microsecond int, fsp int8) Duration { +func NewDuration(hour, minute, second, microsecond int, fsp int) Duration { return Duration{ Duration: gotime.Duration(hour)*gotime.Hour + gotime.Duration(minute)*gotime.Minute + gotime.Duration(second)*gotime.Second + gotime.Duration(microsecond)*gotime.Microsecond, //nolint:durationcheck Fsp: fsp, @@ -1282,11 +1283,11 @@ type Duration struct { gotime.Duration // Fsp is short for Fractional Seconds Precision. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html - Fsp int8 + Fsp int } // MaxMySQLDuration returns Duration with maximum mysql time. -func MaxMySQLDuration(fsp int8) Duration { +func MaxMySQLDuration(fsp int) Duration { return NewDuration(TimeMaxHour, TimeMaxMinute, TimeMaxSecond, 0, fsp) } @@ -1475,14 +1476,14 @@ func (d Duration) ConvertToTime(sc *stmtctx.StatementContext, tp uint8) (Time, e // We will use the “round half up” rule, e.g, >= 0.5 -> 1, < 0.5 -> 0, // so 10:10:10.999999 round 0 -> 10:10:11 // and 10:10:10.000000 round 0 -> 10:10:10 -func (d Duration) RoundFrac(fsp int8, loc *gotime.Location) (Duration, error) { +func (d Duration) RoundFrac(fsp int, loc *gotime.Location) (Duration, error) { tz := loc if tz == nil { logutil.BgLogger().Warn("use gotime.local because sc.timezone is nil") tz = gotime.Local } - fsp, err := CheckFsp(int(fsp)) + fsp, err := CheckFsp(fsp) if err != nil { return d, errors.Trace(err) } @@ -1492,7 +1493,7 @@ func (d Duration) RoundFrac(fsp int8, loc *gotime.Location) (Duration, error) { } n := gotime.Date(0, 0, 0, 0, 0, 0, 0, tz) - nd := n.Add(d.Duration).Round(gotime.Duration(math.Pow10(9-int(fsp))) * gotime.Nanosecond).Sub(n) //nolint:durationcheck + nd := n.Add(d.Duration).Round(gotime.Duration(math.Pow10(9-fsp)) * gotime.Nanosecond).Sub(n) //nolint:durationcheck return Duration{Duration: nd, Fsp: fsp}, nil } @@ -1643,7 +1644,7 @@ func checkHHMMSS(hms [3]int) bool { } // matchFrac returns overflow, fraction, rest, error -func matchFrac(str string, fsp int8) (bool, int, string, error) { +func matchFrac(str string, fsp int) (bool, int, string, error) { rest, err := parser.Char(str, '.') if err != nil { return false, 0, str, nil @@ -1662,8 +1663,8 @@ func matchFrac(str string, fsp int8) (bool, int, string, error) { return overflow, frac, rest, nil } -func matchDuration(str string, fsp int8) (Duration, error) { - fsp, err := CheckFsp(int(fsp)) +func matchDuration(str string, fsp int) (Duration, error) { + fsp, err := CheckFsp(fsp) if err != nil { return ZeroDuration, errors.Trace(err) } @@ -1760,7 +1761,7 @@ func canFallbackToDateTime(str string) bool { // ParseDuration parses the time form a formatted string with a fractional seconds part, // returns the duration type Time value. // See http://dev.mysql.com/doc/refman/5.7/en/fractional-seconds.html -func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int8) (Duration, error) { +func ParseDuration(sc *stmtctx.StatementContext, str string, fsp int) (Duration, error) { rest := strings.TrimSpace(str) d, err := matchDuration(rest, fsp) if err == nil { @@ -1926,12 +1927,12 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // The valid datetime range is from '1000-01-01 00:00:00.000000' to '9999-12-31 23:59:59.999999'. // The valid timestamp range is from '1970-01-01 00:00:01.000000' to '2038-01-19 03:14:07.999999'. // The valid date range is from '1000-01-01' to '9999-12-31' -func ParseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int8) (Time, error) { +func ParseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int) (Time, error) { return parseTime(sc, str, tp, fsp, false) } // ParseTimeFromFloatString is similar to ParseTime, except that it's used to parse a float converted string. -func ParseTimeFromFloatString(sc *stmtctx.StatementContext, str string, tp byte, fsp int8) (Time, error) { +func ParseTimeFromFloatString(sc *stmtctx.StatementContext, str string, tp byte, fsp int) (Time, error) { // MySQL compatibility: 0.0 should not be converted to null, see #11203 if len(str) >= 3 && str[:3] == "0.0" { return NewTime(ZeroCoreTime, tp, DefaultFsp), nil @@ -1939,8 +1940,8 @@ func ParseTimeFromFloatString(sc *stmtctx.StatementContext, str string, tp byte, return parseTime(sc, str, tp, fsp, true) } -func parseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int8, isFloat bool) (Time, error) { - fsp, err := CheckFsp(int(fsp)) +func parseTime(sc *stmtctx.StatementContext, str string, tp byte, fsp int, isFloat bool) (Time, error) { + fsp, err := CheckFsp(fsp) if err != nil { return NewTime(ZeroCoreTime, tp, DefaultFsp), errors.Trace(err) } @@ -1986,7 +1987,7 @@ func ParseTimeFromYear(sc *stmtctx.StatementContext, year int64) (Time, error) { // ParseTimeFromNum parses a formatted int64, // returns the value which type is tp. -func ParseTimeFromNum(sc *stmtctx.StatementContext, num int64, tp byte, fsp int8) (Time, error) { +func ParseTimeFromNum(sc *stmtctx.StatementContext, num int64, tp byte, fsp int) (Time, error) { // MySQL compatibility: 0 should not be converted to null, see #11203 if num == 0 { zt := NewTime(ZeroCoreTime, tp, DefaultFsp) @@ -2002,7 +2003,7 @@ func ParseTimeFromNum(sc *stmtctx.StatementContext, num int64, tp byte, fsp int8 } return zt, nil } - fsp, err := CheckFsp(int(fsp)) + fsp, err := CheckFsp(fsp) if err != nil { return NewTime(ZeroCoreTime, tp, DefaultFsp), errors.Trace(err) } @@ -2393,7 +2394,7 @@ func parseTimeValue(format string, index, cnt int) (int64, int64, int64, int64, if err != nil { return 0, 0, 0, 0, ErrWrongValue.GenWithStackByArgs(DateTimeStr, originalFmt) } - microseconds, err := strconv.ParseInt(alignFrac(fields[MicrosecondIndex], int(MaxFsp)), 10, 64) + microseconds, err := strconv.ParseInt(alignFrac(fields[MicrosecondIndex], MaxFsp), 10, 64) if err != nil { return 0, 0, 0, 0, ErrWrongValue.GenWithStackByArgs(DateTimeStr, originalFmt) } diff --git a/types/time_test.go b/types/time_test.go index fac16e0b289d9..3ce53c0353c81 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -34,7 +34,7 @@ func TestTimeEncoding(t *testing.T) { tests := []struct { Year, Month, Day, Hour, Minute, Second, Microsecond int Type uint8 - Fsp int8 + Fsp int Expect uint64 }{ {2019, 9, 16, 0, 0, 0, 0, mysql.TypeDatetime, 0, 0b1111110001110011000000000000000000000000000000000000000000000}, @@ -119,7 +119,7 @@ func TestDateTime(t *testing.T) { fspTbl := []struct { Input string - Fsp int8 + Fsp int Expect string }{ {"20170118.123", 6, "2017-01-18 12:03:00.000000"}, @@ -396,9 +396,9 @@ func TestTime(t *testing.T) { func TestDurationAdd(t *testing.T) { table := []struct { Input string - Fsp int8 + Fsp int InputAdd string - FspAdd int8 + FspAdd int Expect string }{ {"00:00:00.1", 1, "00:00:00.1", 1, "00:00:00.2"}, @@ -434,9 +434,9 @@ func TestDurationSub(t *testing.T) { sc.IgnoreZeroInDate = true table := []struct { Input string - Fsp int8 + Fsp int InputAdd string - FspAdd int8 + FspAdd int Expect string }{ {"00:00:00.1", 1, "00:00:00.1", 1, "00:00:00.0"}, @@ -458,7 +458,7 @@ func TestTimeFsp(t *testing.T) { sc.IgnoreZeroInDate = true table := []struct { Input string - Fsp int8 + Fsp int Expect string }{ {"00:00:00.1", 0, "00:00:00"}, @@ -481,7 +481,7 @@ func TestTimeFsp(t *testing.T) { errTable := []struct { Input string - Fsp int8 + Fsp int }{ {"00:00:00.1", -2}, } @@ -691,7 +691,7 @@ func TestToNumber(t *testing.T) { sc.TimeZone = losAngelesTz tblDateTime := []struct { Input string - Fsp int8 + Fsp int Expect string }{ {"12-12-31 11:30:45", 0, "20121231113045"}, @@ -714,7 +714,7 @@ func TestToNumber(t *testing.T) { // Fix issue #1046 tblDate := []struct { Input string - Fsp int8 + Fsp int Expect string }{ {"12-12-31 11:30:45", 0, "20121231"}, @@ -736,7 +736,7 @@ func TestToNumber(t *testing.T) { tblDuration := []struct { Input string - Fsp int8 + Fsp int Expect string }{ {"11:30:45", 0, "113045"}, @@ -764,7 +764,7 @@ func TestParseTimeFromFloatString(t *testing.T) { sc.IgnoreZeroInDate = true table := []struct { Input string - Fsp int8 + Fsp int ExpectError bool Expect string }{ @@ -794,7 +794,7 @@ func TestParseTimeFromFloatString(t *testing.T) { func TestParseFrac(t *testing.T) { tbl := []struct { S string - Fsp int8 + Fsp int Ret int Overflow bool }{ @@ -833,7 +833,7 @@ func TestRoundFrac(t *testing.T) { sc.TimeZone = time.UTC tbl := []struct { Input string - Fsp int8 + Fsp int Except string }{ {"2012-12-31 11:30:45.123456", 4, "2012-12-31 11:30:45.1235"}, @@ -862,7 +862,7 @@ func TestRoundFrac(t *testing.T) { sc.TimeZone = losAngelesTz tbl = []struct { Input string - Fsp int8 + Fsp int Except string }{ {"2019-11-25 07:25:45.123456", 4, "2019-11-25 07:25:45.1235"}, @@ -884,7 +884,7 @@ func TestRoundFrac(t *testing.T) { tbl = []struct { Input string - Fsp int8 + Fsp int Except string }{ {"11:30:45.123456", 4, "11:30:45.1235"}, @@ -905,7 +905,7 @@ func TestRoundFrac(t *testing.T) { cols := []struct { input time.Time - fsp int8 + fsp int output time.Time }{ {time.Date(2011, 11, 11, 10, 10, 10, 888888, time.UTC), 0, time.Date(2011, 11, 11, 10, 10, 10, 11, time.UTC)}, @@ -926,7 +926,7 @@ func TestConvert(t *testing.T) { sc.TimeZone = losAngelesTz tbl := []struct { Input string - Fsp int8 + Fsp int Except string }{ {"2012-12-31 11:30:45.123456", 4, "11:30:45.1235"}, @@ -948,7 +948,7 @@ func TestConvert(t *testing.T) { tblDuration := []struct { Input string - Fsp int8 + Fsp int }{ {"11:30:45.123456", 4}, {"11:30:45.123456", 6}, @@ -1057,6 +1057,11 @@ func TestParseDateFormat(t *testing.T) { {"T10:10:10", nil}, {"2011-11-11x", []string{"2011", "11", "11x"}}, {"xxx 10:10:10", nil}, + {"2022-02-01\n16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\f16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\v16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\r16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\t16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, } for _, tt := range tbl { @@ -1439,7 +1444,7 @@ func TestExtractDurationValue(t *testing.T) { func TestCurrentTime(t *testing.T) { res := types.CurrentTime(mysql.TypeTimestamp) require.Equal(t, mysql.TypeTimestamp, res.Type()) - require.Equal(t, int8(0), res.Fsp()) + require.Equal(t, 0, res.Fsp()) } func TestInvalidZero(t *testing.T) { @@ -1453,16 +1458,16 @@ func TestInvalidZero(t *testing.T) { func TestGetFsp(t *testing.T) { res := types.GetFsp("2019:04:12 14:00:00.123456") - require.Equal(t, int8(6), res) + require.Equal(t, 6, res) res = types.GetFsp("2019:04:12 14:00:00.1234567890") - require.Equal(t, int8(6), res) + require.Equal(t, 6, res) res = types.GetFsp("2019:04:12 14:00:00.1") - require.Equal(t, int8(1), res) + require.Equal(t, 1, res) res = types.GetFsp("2019:04:12 14:00:00") - require.Equal(t, int8(0), res) + require.Equal(t, 0, res) } func TestExtractDatetimeNum(t *testing.T) { @@ -1788,7 +1793,7 @@ func TestTimeOverflow(t *testing.T) { func TestTruncateFrac(t *testing.T) { cols := []struct { input time.Time - fsp int8 + fsp int output time.Time }{ {time.Date(2011, 11, 11, 10, 10, 10, 888888, time.UTC), 0, time.Date(2011, 11, 11, 10, 10, 10, 11, time.UTC)}, @@ -1957,7 +1962,7 @@ func TestParseWithTimezone(t *testing.T) { // note that sysTZ won't affect the physical time the string literal represents. cases := []struct { lit string - fsp int8 + fsp int gt time.Time sysTZ *time.Location }{ diff --git a/unstable.txt b/unstable.txt new file mode 100644 index 0000000000000..eb30599933f7d --- /dev/null +++ b/unstable.txt @@ -0,0 +1,12 @@ +ddl TestTiFlashReplicaPartitionTableNormal +ddl TestTestSerialStatSuite +ddl testSerialDBSuite.TestModifyColumnTypeWhenInterception +session testPessimisticSuite.TestSelectForUpdateNoWait +server TestGetSchemaStorage +server TestUptime +executor TestSpillToDisk +executor TestHashRowContainer +executor TestExplainAnalyzeCTEMemoryAndDiskInfo +executor TestDefaultValForAnalyze +executor TestPBMemoryLeak +statistics/handle TestAnalyzeGlobalStatsWithOpts1 diff --git a/util/admin/admin.go b/util/admin/admin.go index 863f843fca350..a55fcc02d5aaf 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -19,14 +19,13 @@ import ( "encoding/json" "math" "sort" - "time" + "strings" "github.com/pingcap/errors" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" @@ -36,6 +35,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/logutil/consistency" decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/sqlexec" "go.uber.org/zap" @@ -292,8 +292,8 @@ type RecordData struct { Values []types.Datum } -func getCount(exec sqlexec.RestrictedSQLExecutor, stmt ast.StmtNode, snapshot uint64) (int64, error) { - rows, _, err := exec.ExecRestrictedStmt(context.Background(), stmt, sqlexec.ExecOptionWithSnapshot(snapshot)) +func getCount(exec sqlexec.RestrictedSQLExecutor, snapshot uint64, sql string, args ...interface{}) (int64, error) { + rows, _, err := exec.ExecRestrictedSQL(context.Background(), []sqlexec.OptionFuncAlias{sqlexec.ExecOptionWithSnapshot(snapshot)}, sql, args...) if err != nil { return 0, errors.Trace(err) } @@ -321,12 +321,6 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices defer func() { ctx.GetSessionVars().OptimizerUseInvisibleIndexes = false }() - // Add `` for some names like `table name`. - exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.Background(), true, "SELECT COUNT(*) FROM %n.%n USE INDEX()", dbName, tableName) - if err != nil { - return 0, 0, errors.Trace(err) - } var snapshot uint64 txn, err := ctx.Txn(false) @@ -340,16 +334,14 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices snapshot = ctx.GetSessionVars().SnapshotTS } - tblCnt, err := getCount(exec, stmt, snapshot) + // Add `` for some names like `table name`. + exec := ctx.(sqlexec.RestrictedSQLExecutor) + tblCnt, err := getCount(exec, snapshot, "SELECT COUNT(*) FROM %n.%n USE INDEX()", dbName, tableName) if err != nil { return 0, 0, errors.Trace(err) } for i, idx := range indices { - stmt, err := exec.ParseWithParams(context.Background(), true, "SELECT COUNT(*) FROM %n.%n USE INDEX(%n)", dbName, tableName, idx) - if err != nil { - return 0, i, errors.Trace(err) - } - idxCnt, err := getCount(exec, stmt, snapshot) + idxCnt, err := getCount(exec, snapshot, "SELECT COUNT(*) FROM %n.%n USE INDEX(%n)", dbName, tableName, idx) if err != nil { return 0, i, errors.Trace(err) } @@ -371,13 +363,41 @@ func CheckIndicesCount(ctx sessionctx.Context, dbName, tableName string, indices } // CheckRecordAndIndex is exported for testing. -func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { +func CheckRecordAndIndex(ctx context.Context, sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error { sc := sessCtx.GetSessionVars().StmtCtx cols := make([]*table.Column, len(idx.Meta().Columns)) for i, col := range idx.Meta().Columns { cols[i] = t.Cols()[col.Offset] } + ir := func() *consistency.Reporter { + return &consistency.Reporter{ + HandleEncode: func(handle kv.Handle) kv.Key { + return tablecodec.EncodeRecordKey(t.RecordPrefix(), handle) + }, + IndexEncode: func(idxRow *consistency.RecordData) kv.Key { + var matchingIdx table.Index + for _, v := range t.Indices() { + if strings.EqualFold(v.Meta().Name.String(), idx.Meta().Name.O) { + matchingIdx = v + break + } + } + if matchingIdx == nil { + return nil + } + k, _, err := matchingIdx.GenIndexKey(sessCtx.GetSessionVars().StmtCtx, idxRow.Values, idxRow.Handle, nil) + if err != nil { + return nil + } + return k + }, + Tbl: t.Meta(), + Idx: idx.Meta(), + Sctx: sessCtx, + } + } + startKey := tablecodec.EncodeRecordKey(t.RecordPrefix(), kv.IntHandle(math.MinInt64)) filterFunc := func(h1 kv.Handle, vals1 []types.Datum, cols []*table.Column) (bool, error) { for i, val := range vals1 { @@ -396,16 +416,16 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table } isExist, h2, err := idx.Exist(sc, txn, vals1, h1) if kv.ErrKeyExists.Equal(err) { - record1 := &RecordData{Handle: h1, Values: vals1} - record2 := &RecordData{Handle: h2, Values: vals1} - return false, ErrDataInConsistent.GenWithStackByArgs(record2, record1) + record1 := &consistency.RecordData{Handle: h1, Values: vals1} + record2 := &consistency.RecordData{Handle: h2, Values: vals1} + return false, ir().ReportAdminCheckInconsistent(ctx, h1, record2, record1) } if err != nil { return false, errors.Trace(err) } if !isExist { - record := &RecordData{Handle: h1, Values: vals1} - return false, ErrDataInConsistent.GenWithStackByArgs(nil, record) + record := &consistency.RecordData{Handle: h1, Values: vals1} + return false, ir().ReportAdminCheckInconsistent(ctx, h1, nil, record) } return true, nil @@ -461,7 +481,7 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab return errors.Trace(err) } - rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), time.UTC, nil) + rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), nil) if err != nil { return errors.Trace(err) } @@ -485,8 +505,6 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab } var ( - // ErrDataInConsistent indicate that meets inconsistent data. - ErrDataInConsistent = dbterror.ClassAdmin.NewStd(errno.ErrDataInConsistent) // ErrDDLJobNotFound indicates the job id was not found. ErrDDLJobNotFound = dbterror.ClassAdmin.NewStd(errno.ErrDDLJobNotFound) // ErrCancelFinishedDDLJob returns when cancel a finished ddl job. diff --git a/util/admin/admin_integration_test.go b/util/admin/admin_integration_test.go index a6000c73fecc7..36fed945d391c 100644 --- a/util/admin/admin_integration_test.go +++ b/util/admin/admin_integration_test.go @@ -19,7 +19,9 @@ import ( "testing" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) func TestAdminCheckTable(t *testing.T) { @@ -109,3 +111,36 @@ func TestAdminCheckTableClusterIndex(t *testing.T) { tk.MustExec("insert into t values (1000, '1000', 1000, '1000', '1000');") tk.MustExec("admin check table t;") } + +func TestAdminCheckTableCorrupted(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int, v int, UNIQUE KEY i1(id, v))") + tk.MustExec("begin") + tk.MustExec("insert into t values (1, 1)") + + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + memBuffer := txn.GetMemBuffer() + it, err := memBuffer.Iter(nil, nil) + require.NoError(t, err) + for it.Valid() { + if tablecodec.IsRecordKey(it.Key()) && len(it.Value()) > 0 { + value := make([]byte, len(it.Value())) + key := make([]byte, len(it.Key())) + copy(key, it.Key()) + copy(value, it.Value()) + key[len(key)-1] += 1 + memBuffer.Set(key, value) + } + err = it.Next() + require.NoError(t, err) + } + + tk.MustExec("commit") + err = tk.ExecToErr("admin check table t") + require.Error(t, err) +} diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index 72ff15413d935..591e741e6689b 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -353,7 +353,6 @@ func TestIsJobRollbackable(t *testing.T) { func TestError(t *testing.T) { kvErrs := []*terror.Error{ - ErrDataInConsistent, ErrDDLJobNotFound, ErrCancelFinishedDDLJob, ErrCannotCancelDDLJob, diff --git a/util/admin/main_test.go b/util/admin/main_test.go index e600c43eaefb8..da69646aed519 100644 --- a/util/admin/main_test.go +++ b/util/admin/main_test.go @@ -31,7 +31,7 @@ func TestMain(m *testing.M) { }) opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/util/benchdaily/bench_daily.go b/util/benchdaily/bench_daily.go index 35cec3e832763..32c431159e18e 100644 --- a/util/benchdaily/bench_daily.go +++ b/util/benchdaily/bench_daily.go @@ -82,6 +82,8 @@ func Run(tests ...func(b *testing.B)) { writeBenchResultToFile(res, *outfile) } +// readBenchResultFromFile is used by the daily bench test. +// nolint: unused, deadcode func readBenchResultFromFile(file string) []BenchResult { f, err := os.Open(file) if err != nil { diff --git a/util/chunk/alloc.go b/util/chunk/alloc.go index c2bfae8a4f600..806266cfde788 100644 --- a/util/chunk/alloc.go +++ b/util/chunk/alloc.go @@ -23,7 +23,7 @@ import ( // The typical usage is to call Reset() to recycle objects into a pool, // and Alloc() allocates from the pool. type Allocator interface { - Alloc(fields []*types.FieldType, cap, maxChunkSize int) *Chunk + Alloc(fields []*types.FieldType, capacity, maxChunkSize int) *Chunk Reset() } @@ -48,7 +48,7 @@ type allocator struct { } // Alloc implements the Allocator interface. -func (a *allocator) Alloc(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { +func (a *allocator) Alloc(fields []*types.FieldType, capacity, maxChunkSize int) *Chunk { var chk *Chunk // Try to alloc from the free list. if len(a.free) > 0 { @@ -59,7 +59,7 @@ func (a *allocator) Alloc(fields []*types.FieldType, cap, maxChunkSize int) *Chu } // Init the chunk fields. - chk.capacity = mathutil.Min(cap, maxChunkSize) + chk.capacity = mathutil.Min(capacity, maxChunkSize) chk.requiredRows = maxChunkSize // Allocate the chunk columns from the pool column allocator. for _, f := range fields { @@ -116,6 +116,9 @@ func (alloc *poolColumnAllocator) init() { } func (alloc *poolColumnAllocator) put(col *Column) { + if col.avoidReusing { + return + } typeSize := col.typeSize() if typeSize <= 0 { return diff --git a/util/chunk/alloc_test.go b/util/chunk/alloc_test.go index 59c097b09d1bd..3f19db3574b07 100644 --- a/util/chunk/alloc_test.go +++ b/util/chunk/alloc_test.go @@ -114,7 +114,7 @@ func TestColumnAllocator(t *testing.T) { // Check max column size. freeList := alloc1.pool[getFixedLen(ft)] require.NotNil(t, freeList) - require.Equal(t, len(freeList), maxFreeColumnsPerType) + require.Len(t, freeList, maxFreeColumnsPerType) } func TestNoDuplicateColumnReuse(t *testing.T) { @@ -151,3 +151,54 @@ func TestNoDuplicateColumnReuse(t *testing.T) { } } } + +func TestAvoidColumnReuse(t *testing.T) { + // For issue: https://github.com/pingcap/tidb/issues/31981 + // Some chunk columns are references to rpc message. + // So when reusing Chunk, we should ignore them. + + fieldTypes := []*types.FieldType{ + {Tp: mysql.TypeVarchar}, + {Tp: mysql.TypeJSON}, + {Tp: mysql.TypeFloat}, + {Tp: mysql.TypeNewDecimal}, + {Tp: mysql.TypeDouble}, + {Tp: mysql.TypeLonglong}, + {Tp: mysql.TypeTimestamp}, + {Tp: mysql.TypeDatetime}, + } + alloc := NewAllocator() + for i := 0; i < maxFreeChunks+10; i++ { + chk := alloc.Alloc(fieldTypes, 5, 10) + for _, col := range chk.columns { + col.avoidReusing = true + } + } + alloc.Reset() + + a := alloc.columnAlloc + // Make sure no duplicated column in the pool. + for _, p := range a.pool { + require.True(t, p.empty()) + } + + // test decoder will set avoid reusing flag. + chk := alloc.Alloc(fieldTypes, 5, 1024) + for i := 0; i <= 10; i++ { + for _, col := range chk.columns { + col.AppendNull() + } + } + codec := &Codec{fieldTypes} + buf := codec.Encode(chk) + + decoder := NewDecoder( + NewChunkWithCapacity(fieldTypes, 0), + fieldTypes, + ) + decoder.Reset(buf) + decoder.ReuseIntermChk(chk) + for _, col := range chk.columns { + require.True(t, col.avoidReusing) + } +} diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index 40a2b059510ac..710d52ed75a4a 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -57,17 +57,17 @@ const ( ) // NewChunkWithCapacity creates a new chunk with field types and capacity. -func NewChunkWithCapacity(fields []*types.FieldType, cap int) *Chunk { - return New(fields, cap, cap) +func NewChunkWithCapacity(fields []*types.FieldType, capacity int) *Chunk { + return New(fields, capacity, capacity) } // New creates a new chunk. // cap: the limit for the max number of rows. // maxChunkSize: the max limit for the number of rows. -func New(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { +func New(fields []*types.FieldType, capacity, maxChunkSize int) *Chunk { chk := &Chunk{ columns: make([]*Column, 0, len(fields)), - capacity: mathutil.Min(cap, maxChunkSize), + capacity: mathutil.Min(capacity, maxChunkSize), // set the default value of requiredRows to maxChunkSize to let chk.IsFull() behave // like how we judge whether a chunk is full now, then the statement // "chk.NumRows() < maxChunkSize" @@ -83,14 +83,14 @@ func New(fields []*types.FieldType, cap, maxChunkSize int) *Chunk { // renewWithCapacity creates a new Chunk based on an existing Chunk with capacity. The newly // created Chunk has the same data schema with the old Chunk. -func renewWithCapacity(chk *Chunk, cap, requiredRows int) *Chunk { +func renewWithCapacity(chk *Chunk, capacity, requiredRows int) *Chunk { if chk.columns == nil { return &Chunk{} } return &Chunk{ - columns: renewColumns(chk.columns, cap), + columns: renewColumns(chk.columns, capacity), numVirtualRows: 0, - capacity: cap, + capacity: capacity, requiredRows: requiredRows, } } @@ -107,10 +107,10 @@ func Renew(chk *Chunk, maxChunkSize int) *Chunk { // renewColumns creates the columns of a Chunk. The capacity of the newly // created columns is equal to cap. -func renewColumns(oldCol []*Column, cap int) []*Column { +func renewColumns(oldCol []*Column, capacity int) []*Column { columns := make([]*Column, 0, len(oldCol)) for _, col := range oldCol { - columns = append(columns, newColumn(col.typeSize(), cap)) + columns = append(columns, newColumn(col.typeSize(), capacity)) } return columns } diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index bb7cb9f9b47c0..69f87c6634ce0 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -333,13 +333,13 @@ func newChunk(elemLen ...int) *Chunk { return chk } -func newChunkWithInitCap(cap int, elemLen ...int) *Chunk { +func newChunkWithInitCap(capacity int, elemLen ...int) *Chunk { chk := &Chunk{} for _, l := range elemLen { if l > 0 { - chk.columns = append(chk.columns, newFixedLenColumn(l, cap)) + chk.columns = append(chk.columns, newFixedLenColumn(l, capacity)) } else { - chk.columns = append(chk.columns, newVarLenColumn(cap)) + chk.columns = append(chk.columns, newVarLenColumn(capacity)) } } return chk diff --git a/util/chunk/codec.go b/util/chunk/codec.go index ca872e7797bb9..9f2d4bb948d82 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -140,6 +140,9 @@ func (c *Codec) decodeColumn(buffer []byte, col *Column, ordinal int) (remained // decode data. col.data = buffer[:numDataBytes:numDataBytes] + // The column reference the data of the grpc response, the memory of the grpc message cannot be GCed if we reuse + // this column. Thus, we set `avoidReusing` to true. + col.avoidReusing = true return buffer[numDataBytes:] } diff --git a/util/chunk/column.go b/util/chunk/column.go index 2f8795e1ed06c..6fca6c89eecd0 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -66,55 +66,57 @@ type Column struct { offsets []int64 // used for varLen column. Row i starts from data[offsets[i]] data []byte elemBuf []byte + + avoidReusing bool // avoid reusing the Column by allocator } // ColumnAllocator defines an allocator for Column. type ColumnAllocator interface { - NewColumn(ft *types.FieldType, cap int) *Column + NewColumn(ft *types.FieldType, count int) *Column } // DefaultColumnAllocator is the default implementation of ColumnAllocator. type DefaultColumnAllocator struct{} // NewColumn implements the ColumnAllocator interface. -func (DefaultColumnAllocator) NewColumn(ft *types.FieldType, cap int) *Column { - return newColumn(getFixedLen(ft), cap) +func (DefaultColumnAllocator) NewColumn(ft *types.FieldType, capacity int) *Column { + return newColumn(getFixedLen(ft), capacity) } // NewColumn creates a new column with the specific type and capacity. -func NewColumn(ft *types.FieldType, cap int) *Column { - return newColumn(getFixedLen(ft), cap) +func NewColumn(ft *types.FieldType, capacity int) *Column { + return newColumn(getFixedLen(ft), capacity) } -func newColumn(typeSize, cap int) *Column { +func newColumn(ts, capacity int) *Column { var col *Column - if typeSize == varElemLen { - col = newVarLenColumn(cap) + if ts == varElemLen { + col = newVarLenColumn(capacity) } else { - col = newFixedLenColumn(typeSize, cap) + col = newFixedLenColumn(ts, capacity) } return col } // newFixedLenColumn creates a fixed length Column with elemLen and initial data capacity. -func newFixedLenColumn(elemLen, cap int) *Column { +func newFixedLenColumn(elemLen, capacity int) *Column { return &Column{ elemBuf: make([]byte, elemLen), - data: make([]byte, 0, cap*elemLen), - nullBitmap: make([]byte, 0, (cap+7)>>3), + data: make([]byte, 0, capacity*elemLen), + nullBitmap: make([]byte, 0, (capacity+7)>>3), } } // newVarLenColumn creates a variable length Column with initial data capacity. -func newVarLenColumn(cap int) *Column { +func newVarLenColumn(capacity int) *Column { estimatedElemLen := 8 // For varLenColumn (e.g. varchar), the accurate length of an element is unknown. // Therefore, in the first executor.Next we use an experience value -- 8 (so it may make runtime.growslice) return &Column{ - offsets: make([]int64, 1, cap+1), - data: make([]byte, 0, cap*estimatedElemLen), - nullBitmap: make([]byte, 0, (cap+7)>>3), + offsets: make([]int64, 1, capacity+1), + data: make([]byte, 0, capacity*estimatedElemLen), + nullBitmap: make([]byte, 0, (capacity+7)>>3), } } @@ -596,7 +598,7 @@ func (c *Column) GetTime(rowID int) types.Time { // GetDuration returns the Duration in the specific row. func (c *Column) GetDuration(rowID int, fillFsp int) types.Duration { dur := *(*int64)(unsafe.Pointer(&c.data[rowID*8])) - return types.Duration{Duration: time.Duration(dur), Fsp: int8(fillFsp)} + return types.Duration{Duration: time.Duration(dur), Fsp: fillFsp} } func (c *Column) getNameValue(rowID int) (string, uint64) { diff --git a/util/chunk/column_test.go b/util/chunk/column_test.go index 5f89f8a9659a6..7e53f530c33f0 100644 --- a/util/chunk/column_test.go +++ b/util/chunk/column_test.go @@ -75,7 +75,7 @@ func TestColumnCopyReconstructFixedLen(t *testing.T) { } } require.Equal(t, col.nullCount(), nullCnt) - require.Equal(t, len(sel), col.length) + require.Len(t, sel, col.length) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -85,7 +85,7 @@ func TestColumnCopyReconstructFixedLen(t *testing.T) { } } - require.Equal(t, len(sel)+128, col.length) + require.Len(t, sel, col.length-128) require.Equal(t, nullCnt+128/2, col.nullCount()) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -131,7 +131,7 @@ func TestColumnCopyReconstructVarLen(t *testing.T) { } } require.Equal(t, col.nullCount(), nullCnt) - require.Equal(t, len(sel), col.length) + require.Len(t, sel, col.length) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -141,7 +141,7 @@ func TestColumnCopyReconstructVarLen(t *testing.T) { } } - require.Equal(t, len(sel)+128, col.length) + require.Len(t, sel, col.length-128) require.Equal(t, nullCnt+128/2, col.nullCount()) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -465,7 +465,7 @@ func TestReconstructFixedLen(t *testing.T) { } } require.Equal(t, col.nullCount(), nullCnt) - require.Equal(t, len(sel), col.length) + require.Len(t, sel, col.length) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -475,7 +475,7 @@ func TestReconstructFixedLen(t *testing.T) { } } - require.Equal(t, len(sel)+128, col.length) + require.Len(t, sel, col.length-128) require.Equal(t, nullCnt+128/2, col.nullCount()) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -521,7 +521,7 @@ func TestReconstructVarLen(t *testing.T) { } } require.Equal(t, col.nullCount(), nullCnt) - require.Equal(t, len(sel), col.length) + require.Len(t, sel, col.length) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -531,7 +531,7 @@ func TestReconstructVarLen(t *testing.T) { } } - require.Equal(t, len(sel)+128, col.length) + require.Len(t, sel, col.length-128) require.Equal(t, nullCnt+128/2, col.nullCount()) for i := 0; i < 128; i++ { if i%2 == 0 { @@ -680,7 +680,7 @@ func TestSetNulls(t *testing.T) { } col.SetNulls(begin, end, true) - require.Equal(t, len(nullMap), col.nullCount()) + require.Len(t, nullMap, col.nullCount()) for k := range nullMap { require.True(t, col.IsNull(k)) } @@ -780,7 +780,7 @@ func TestResize(t *testing.T) { col = NewColumn(types.NewFieldType(mysql.TypeDuration), 1024) for i := 0; i < 1024; i++ { - col.AppendDuration(types.Duration{Duration: time.Duration(i), Fsp: int8(i)}) + col.AppendDuration(types.Duration{Duration: time.Duration(i), Fsp: i}) } col.ResizeGoDuration(1024, false) for i := 0; i < 1024; i++ { diff --git a/util/chunk/disk.go b/util/chunk/disk.go index 605f72d63e911..516d335149b54 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -99,34 +99,6 @@ func (l *ListInDisk) GetDiskTracker() *disk.Tracker { return l.diskTracker } -// flush empties the write buffer, please call flush before read! -func (l *ListInDisk) flush() (err error) { - // buffered is not zero only after Add and before GetRow, after the first flush, buffered will always be zero, - // hence we use a RWLock to allow quicker quit. - l.bufFlushMutex.RLock() - checksumWriter := l.w - l.bufFlushMutex.RUnlock() - if checksumWriter == nil { - return nil - } - l.bufFlushMutex.Lock() - defer l.bufFlushMutex.Unlock() - if l.w != nil { - err = l.w.Close() - if err != nil { - return - } - l.w = nil - // the l.disk is the underlying object of the l.w, it will be closed - // after calling l.w.Close, we need to reopen it before reading rows. - l.disk, err = os.Open(l.disk.Name()) - if err != nil { - return errors2.Trace(err) - } - } - return -} - // Add adds a chunk to the ListInDisk. Caller must make sure the input chk // is not empty and not used any more and has the same field types. // Warning: do not mix Add and GetRow (always use GetRow after you have added all the chunks), and do not use Add concurrently. diff --git a/util/chunk/disk_test.go b/util/chunk/disk_test.go index fc474f11494df..c22b525477a02 100644 --- a/util/chunk/disk_test.go +++ b/util/chunk/disk_test.go @@ -26,6 +26,7 @@ import ( "testing" "github.com/cznic/mathutil" + errors2 "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" @@ -172,6 +173,33 @@ func (l *listInDiskWriteDisk) GetRow(ptr RowPtr) (row Row, err error) { return row, err } +func (l *listInDiskWriteDisk) flush() (err error) { + // buffered is not zero only after Add and before GetRow, after the first flush, buffered will always be zero, + // hence we use a RWLock to allow quicker quit. + l.bufFlushMutex.RLock() + checksumWriter := l.w + l.bufFlushMutex.RUnlock() + if checksumWriter == nil { + return nil + } + l.bufFlushMutex.Lock() + defer l.bufFlushMutex.Unlock() + if l.w != nil { + err = l.w.Close() + if err != nil { + return + } + l.w = nil + // the l.disk is the underlying object of the l.w, it will be closed + // after calling l.w.Close, we need to reopen it before reading rows. + l.disk, err = os.Open(l.disk.Name()) + if err != nil { + return errors2.Trace(err) + } + } + return +} + func checkRow(t *testing.T, row1, row2 Row) { require.Equal(t, row2.GetString(0), row1.GetString(0)) require.Equal(t, row2.GetInt64(1), row1.GetInt64(1)) @@ -356,6 +384,6 @@ func testReaderWithCacheNoFlush(t *testing.T) { // Offset is 8, because we want to ignore col length. readCnt, err := checksumReader.ReadAt(data, 8) require.Equal(t, io.EOF, err) - require.Equal(t, len(testData), readCnt) + require.Len(t, testData, readCnt) require.Equal(t, []byte(testData), data[:10]) } diff --git a/util/chunk/iterator.go b/util/chunk/iterator.go index 9b253b4bc576a..ed9905b916a93 100644 --- a/util/chunk/iterator.go +++ b/util/chunk/iterator.go @@ -72,8 +72,8 @@ func (it *iterator4Slice) Begin() Row { // Next implements the Iterator interface. func (it *iterator4Slice) Next() Row { - if len := it.Len(); it.cursor >= len { - it.cursor = len + 1 + if l := it.Len(); it.cursor >= l { + it.cursor = l + 1 return it.End() } row := it.rows[it.cursor] @@ -274,8 +274,8 @@ func (it *iterator4RowPtr) Begin() Row { // Next implements the Iterator interface. func (it *iterator4RowPtr) Next() Row { - if len := it.Len(); it.cursor >= len { - it.cursor = len + 1 + if l := it.Len(); it.cursor >= l { + it.cursor = l + 1 return it.End() } row := it.li.GetRow(it.ptrs[it.cursor]) diff --git a/util/chunk/iterator_test.go b/util/chunk/iterator_test.go index 85e59d23ba153..5537f9c325898 100644 --- a/util/chunk/iterator_test.go +++ b/util/chunk/iterator_test.go @@ -43,7 +43,7 @@ func TestIteratorOnSel(t *testing.T) { } func checkEqual(it Iterator, exp []int64, t *testing.T) { - require.Equal(t, len(exp), it.Len()) + require.Len(t, exp, it.Len()) for row, i := it.Begin(), 0; row != it.End(); row, i = it.Next(), i+1 { require.Equal(t, exp[i], row.GetInt64(0)) } diff --git a/util/chunk/mutrow_test.go b/util/chunk/mutrow_test.go index 32bef71484f7d..318da3d355a68 100644 --- a/util/chunk/mutrow_test.go +++ b/util/chunk/mutrow_test.go @@ -35,7 +35,7 @@ func TestMutRow(t *testing.T) { d := row.GetDatum(i, allTypes[i]) d2 := types.NewDatum(val) cmp, err := d.Compare(sc, &d2, collate.GetCollator(allTypes[i].Collate)) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, 0, cmp) } diff --git a/util/chunk/pool_test.go b/util/chunk/pool_test.go index 02c9e319b1ab1..2bf3752fd5a72 100644 --- a/util/chunk/pool_test.go +++ b/util/chunk/pool_test.go @@ -49,7 +49,7 @@ func TestPoolGetChunk(t *testing.T) { chk := pool.GetChunk(fieldTypes) require.NotNil(t, chk) - require.Equal(t, len(fieldTypes), chk.NumCols()) + require.Len(t, fieldTypes, chk.NumCols()) require.Nil(t, chk.columns[0].elemBuf) require.Nil(t, chk.columns[1].elemBuf) require.Equal(t, getFixedLen(fieldTypes[2]), len(chk.columns[2].elemBuf)) diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index 2bf8ef26644bc..3803a1451df7a 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -226,9 +226,9 @@ func TestSpillActionDeadLock(t *testing.T) { // Goroutine 2: ------------------> SpillDiskAction -> new Goroutine to spill -> ------------------ // new Goroutine created by 2: ---> rc.SpillToDisk (Lock) // In golang, RLock will be blocked after try to get Lock. So it will cause deadlock. - require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock", "return(true)")) defer func() { - require.Nil(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/chunk/testRowContainerDeadLock")) }() sz := 4 fields := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} diff --git a/util/codec/bytes_test.go b/util/codec/bytes_test.go index eb2dfee922aa0..e1a6fc621a754 100644 --- a/util/codec/bytes_test.go +++ b/util/codec/bytes_test.go @@ -57,7 +57,7 @@ func TestBytesCodec(t *testing.T) { } for _, input := range inputs { - require.Equal(t, len(input.dec), EncodedBytesLength(len(input.enc))) + require.Len(t, input.dec, EncodedBytesLength(len(input.enc))) if input.desc { b := EncodeBytesDesc(nil, input.enc) diff --git a/util/codec/codec.go b/util/codec/codec.go index 454e9dd28a9c9..7168bb4449c38 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -116,9 +116,9 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab err = sc.HandleOverflow(err, err) } case types.KindMysqlEnum: - b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlEnum().ToNumber()), comparable) + b = encodeUnsignedInt(b, vals[i].GetMysqlEnum().Value, comparable) case types.KindMysqlSet: - b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlSet().ToNumber()), comparable) + b = encodeUnsignedInt(b, vals[i].GetMysqlSet().Value, comparable) case types.KindMysqlBit, types.KindBinaryLiteral: // We don't need to handle errors here since the literal is ensured to be able to store in uint64 in convertToMysqlBit. var val uint64 @@ -159,9 +159,9 @@ func EstimateValueSize(sc *stmtctx.StatementContext, val types.Datum) (int, erro case types.KindMysqlDecimal: l = valueSizeOfDecimal(val.GetMysqlDecimal(), val.Length(), val.Frac()) + 1 case types.KindMysqlEnum: - l = valueSizeOfUnsignedInt(uint64(val.GetMysqlEnum().ToNumber())) + l = valueSizeOfUnsignedInt(val.GetMysqlEnum().Value) case types.KindMysqlSet: - l = valueSizeOfUnsignedInt(uint64(val.GetMysqlSet().ToNumber())) + l = valueSizeOfUnsignedInt(val.GetMysqlSet().Value) case types.KindMysqlBit, types.KindBinaryLiteral: val, err := val.GetBinaryLiteral().ToInt(sc) terror.Log(errors.Trace(err)) @@ -355,11 +355,11 @@ func encodeHashChunkRowIdx(sc *stmtctx.StatementContext, row chunk.Row, tp *type case mysql.TypeEnum: if mysql.HasEnumSetAsIntFlag(tp.Flag) { flag = uvarintFlag - v := uint64(row.GetEnum(idx).ToNumber()) + v := row.GetEnum(idx).Value b = (*[sizeUint64]byte)(unsafe.Pointer(&v))[:] } else { flag = compactBytesFlag - v := uint64(row.GetEnum(idx).ToNumber()) + v := row.GetEnum(idx).Value str := "" if enum, err := types.ParseEnumValue(tp.Elems, v); err == nil { // str will be empty string if v out of definition of enum. @@ -570,11 +570,11 @@ func HashChunkSelected(sc *stmtctx.StatementContext, h []hash.Hash64, chk *chunk isNull[i] = !ignoreNull } else if mysql.HasEnumSetAsIntFlag(tp.Flag) { buf[0] = uvarintFlag - v := uint64(column.GetEnum(i).ToNumber()) + v := column.GetEnum(i).Value b = (*[sizeUint64]byte)(unsafe.Pointer(&v))[:] } else { buf[0] = compactBytesFlag - v := uint64(column.GetEnum(i).ToNumber()) + v := column.GetEnum(i).Value str := "" if enum, err := types.ParseEnumValue(tp.Elems, v); err == nil { // str will be empty string if v out of definition of enum. @@ -1120,7 +1120,7 @@ func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (re if err != nil { return nil, errors.Trace(err) } - v := types.Duration{Duration: time.Duration(r), Fsp: int8(ft.Decimal)} + v := types.Duration{Duration: time.Duration(r), Fsp: ft.Decimal} chk.AppendDuration(colIdx, v) case jsonFlag: var size int @@ -1144,7 +1144,7 @@ func (decoder *Decoder) DecodeOne(b []byte, colIdx int, ft *types.FieldType) (re func appendIntToChunk(val int64, chk *chunk.Chunk, colIdx int, ft *types.FieldType) { switch ft.Tp { case mysql.TypeDuration: - v := types.Duration{Duration: time.Duration(val), Fsp: int8(ft.Decimal)} + v := types.Duration{Duration: time.Duration(val), Fsp: ft.Decimal} chk.AppendDuration(colIdx, v) default: chk.AppendInt64(colIdx, val) @@ -1154,7 +1154,7 @@ func appendIntToChunk(val int64, chk *chunk.Chunk, colIdx int, ft *types.FieldTy func appendUintToChunk(val uint64, chk *chunk.Chunk, colIdx int, ft *types.FieldType, loc *time.Location) error { switch ft.Tp { case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: - t := types.NewTime(types.ZeroCoreTime, ft.Tp, int8(ft.Decimal)) + t := types.NewTime(types.ZeroCoreTime, ft.Tp, ft.Decimal) var err error err = t.FromPackedUint(val) if err != nil { @@ -1326,9 +1326,9 @@ func HashCode(b []byte, d types.Datum) []byte { decStr := d.GetMysqlDecimal().ToString() b = encodeBytes(b, decStr, false) case types.KindMysqlEnum: - b = encodeUnsignedInt(b, uint64(d.GetMysqlEnum().ToNumber()), false) + b = encodeUnsignedInt(b, d.GetMysqlEnum().Value, false) case types.KindMysqlSet: - b = encodeUnsignedInt(b, uint64(d.GetMysqlSet().ToNumber()), false) + b = encodeUnsignedInt(b, d.GetMysqlSet().Value, false) case types.KindMysqlBit, types.KindBinaryLiteral: val := d.GetBinaryLiteral() b = encodeBytes(b, val, false) diff --git a/util/codec/collation_test.go b/util/codec/collation_test.go index cf64c9ff570be..5c361069c0807 100644 --- a/util/codec/collation_test.go +++ b/util/codec/collation_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/collate" "github.com/stretchr/testify/require" ) @@ -46,8 +45,6 @@ func prepareCollationData() (int, *chunk.Chunk, *chunk.Chunk) { } func TestHashGroupKeyCollation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) sc := &stmtctx.StatementContext{TimeZone: time.Local} tp := types.NewFieldType(mysql.TypeString) n, chk1, chk2 := prepareCollationData() @@ -85,8 +82,6 @@ func TestHashGroupKeyCollation(t *testing.T) { } func TestHashChunkRowCollation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) sc := &stmtctx.StatementContext{TimeZone: time.Local} tp := types.NewFieldType(mysql.TypeString) tps := []*types.FieldType{tp} @@ -129,8 +124,6 @@ func TestHashChunkRowCollation(t *testing.T) { } func TestHashChunkColumnsCollation(t *testing.T) { - collate.SetNewCollationEnabledForTest(true) - defer collate.SetNewCollationEnabledForTest(false) sc := &stmtctx.StatementContext{TimeZone: time.Local} tp := types.NewFieldType(mysql.TypeString) n, chk1, chk2 := prepareCollationData() diff --git a/util/collate/collate.go b/util/collate/collate.go index 1b31abcb62ffb..171f7f7cfe491 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -78,11 +78,6 @@ type WildcardPattern interface { DoMatch(str string) bool } -// EnableNewCollations enables the new collation. -func EnableNewCollations() { - SetNewCollationEnabledForTest(true) -} - // SetNewCollationEnabledForTest sets if the new collation are enabled in test. // Note: Be careful to use this function, if this functions is used in tests, make sure the tests are serial. func SetNewCollationEnabledForTest(flag bool) { @@ -371,6 +366,9 @@ func ProtoToCollation(c int32) string { } func init() { + // Set it to 1 in init() to make sure the tests enable the new collation, it would be covered in bootstrap(). + newCollationEnabled = 1 + newCollatorMap = make(map[string]Collator) newCollatorIDMap = make(map[int]Collator) diff --git a/util/cpuprofile/cpuprofile.go b/util/cpuprofile/cpuprofile.go index 6ef9138ba1f08..752e51eb188a1 100644 --- a/util/cpuprofile/cpuprofile.go +++ b/util/cpuprofile/cpuprofile.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" ) // DefProfileDuration exports for testing. @@ -57,6 +58,7 @@ func StopCPUProfiler() { // Normally, the registered ProfileConsumer will receive the cpu profile data per second. // If the ProfileConsumer (channel) is full, the latest cpu profile data will not be sent to it. // This function is thread-safe. +// WARN: ProfileConsumer should not be closed before unregister. func Register(ch ProfileConsumer) { globalCPUProfiler.register(ch) } @@ -208,6 +210,13 @@ func (p *parallelCPUProfiler) consumersCount() int { func (p *parallelCPUProfiler) sendToConsumers() { p.Lock() + defer func() { + p.Unlock() + if r := recover(); r != nil { + logutil.BgLogger().Error("parallel cpu profiler panic", zap.Any("recover", r)) + } + }() + for c := range p.cs { select { case c <- p.profileData: @@ -216,5 +225,4 @@ func (p *parallelCPUProfiler) sendToConsumers() { } } p.profileData = nil - p.Unlock() } diff --git a/util/cpuprofile/cpuprofile_test.go b/util/cpuprofile/cpuprofile_test.go index 30903d38f2f75..2b7b3ca14a068 100644 --- a/util/cpuprofile/cpuprofile_test.go +++ b/util/cpuprofile/cpuprofile_test.go @@ -177,13 +177,13 @@ func TestGetCPUProfile(t *testing.T) { defer cancel() testutil.MockCPULoad(ctx, "sql", "sql_digest", "plan_digest") var wg sync.WaitGroup - for i := 0; i < 3; i++ { + for i := 0; i < 10; i++ { wg.Add(1) go func() { defer wg.Done() var err error buf := bytes.NewBuffer(nil) - err = getCPUProfile(time.Millisecond*400, buf) + err = getCPUProfile(time.Millisecond*1000, buf) require.NoError(t, err) profileData, err := profile.Parse(buf) require.NoError(t, err) diff --git a/util/disk/tempDir_test.go b/util/disk/tempDir_test.go index 110d22e0bebd6..09e2a87c66709 100644 --- a/util/disk/tempDir_test.go +++ b/util/disk/tempDir_test.go @@ -24,13 +24,12 @@ import ( ) func TestRemoveDir(t *testing.T) { - path, err := os.MkdirTemp("", "tmp-storage-disk-pkg") - require.NoError(t, err) + path := t.TempDir() defer config.RestoreFunc() config.UpdateGlobal(func(conf *config.Config) { conf.TempStoragePath = path }) - err = os.RemoveAll(path) // clean the uncleared temp file during the last run. + err := os.RemoveAll(path) // clean the uncleared temp file during the last run. require.NoError(t, err) err = os.MkdirAll(path, 0755) require.NoError(t, err) diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index a3a763a271fc7..190d42322e4da 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -294,7 +294,7 @@ func TestFormatDurationForExplain(t *testing.T) { } for _, ca := range cases { d, err := time.ParseDuration(ca.t) - require.Nil(t, err) + require.NoError(t, err) result := FormatDuration(d) require.Equal(t, ca.s, result) diff --git a/util/gcutil/gcutil.go b/util/gcutil/gcutil.go index 5d0949f162747..8c60534f9c265 100644 --- a/util/gcutil/gcutil.go +++ b/util/gcutil/gcutil.go @@ -72,11 +72,7 @@ func ValidateSnapshotWithGCSafePoint(snapshotTS, safePointTS uint64) error { // GetGCSafePoint loads GC safe point time from mysql.tidb. func GetGCSafePoint(ctx sessionctx.Context) (uint64, error) { exec := ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.Background(), true, selectVariableValueSQL, "tikv_gc_safe_point") - if err != nil { - return 0, errors.Trace(err) - } - rows, _, err := exec.ExecRestrictedStmt(context.Background(), stmt) + rows, _, err := exec.ExecRestrictedSQL(context.Background(), nil, selectVariableValueSQL, "tikv_gc_safe_point") if err != nil { return 0, errors.Trace(err) } diff --git a/util/keydecoder/main_test.go b/util/keydecoder/main_test.go index 1dd3d666b6a67..81b0bc7afdc80 100644 --- a/util/keydecoder/main_test.go +++ b/util/keydecoder/main_test.go @@ -23,7 +23,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/util/logutil/consistency/reporter.go b/util/logutil/consistency/reporter.go new file mode 100644 index 0000000000000..db14c25a865cf --- /dev/null +++ b/util/logutil/consistency/reporter.go @@ -0,0 +1,293 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package consistency + +import ( + "context" + "encoding/hex" + "encoding/json" + "fmt" + "strconv" + "strings" + "time" + + "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/store/helper" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/dbterror" + "github.com/pingcap/tidb/util/logutil" + "github.com/tikv/client-go/v2/tikv" + "go.uber.org/zap" +) + +var ( + // ErrAdminCheckInconsistent returns for data inconsistency for admin check. + ErrAdminCheckInconsistent = dbterror.ClassAdmin.NewStd(errno.ErrDataInconsistent) + // ErrLookupInconsistent returns for data inconsistency for index lookup. + ErrLookupInconsistent = dbterror.ClassExecutor.NewStd(errno.ErrDataInconsistentMismatchCount) + // ErrAdminCheckInconsistentWithColInfo returns for data inconsistency for admin check but with column info. + ErrAdminCheckInconsistentWithColInfo = dbterror.ClassExecutor.NewStd(errno.ErrDataInconsistentMismatchIndex) +) + +// GetMvccByKey gets the MVCC value by key, and returns a json string including decoded data +func GetMvccByKey(tikvStore helper.Storage, key kv.Key, decodeMvccFn func(kv.Key, *kvrpcpb.MvccGetByKeyResponse, map[string]interface{})) string { + if key == nil { + return "" + } + h := helper.NewHelper(tikvStore) + data, err := h.GetMvccByEncodedKey(key) + if err != nil { + return "" + } + regionID := getRegionIDByKey(tikvStore, key) + + decodeKey := strings.ToUpper(hex.EncodeToString(key)) + + resp := map[string]interface{}{ + "key": decodeKey, + "regionID": regionID, + "mvcc": data, + } + + if decodeMvccFn != nil { + decodeMvccFn(key, data, resp) + } + + rj, err := json.Marshal(resp) + if err != nil { + return "" + } + const maxMvccInfoLen = 5000 + s := string(rj) + if len(s) > maxMvccInfoLen { + s = s[:maxMvccInfoLen] + "[truncated]..." + } + + return s +} + +func getRegionIDByKey(tikvStore helper.Storage, encodedKey []byte) uint64 { + keyLocation, err := tikvStore.GetRegionCache().LocateKey(tikv.NewBackofferWithVars(context.Background(), 500, nil), encodedKey) + if err != nil { + return 0 + } + return keyLocation.Region.GetID() +} + +// Reporter is a helper to generate report. +type Reporter struct { + HandleEncode func(handle kv.Handle) kv.Key + IndexEncode func(idxRow *RecordData) kv.Key + Tbl *model.TableInfo + Idx *model.IndexInfo + Sctx sessionctx.Context +} + +// DecodeRowMvccData creates a closure that captures the tableInfo to be used a decode function in GetMvccByKey. +func DecodeRowMvccData(tableInfo *model.TableInfo) func(kv.Key, *kvrpcpb.MvccGetByKeyResponse, map[string]interface{}) { + return func(key kv.Key, respValue *kvrpcpb.MvccGetByKeyResponse, outMap map[string]interface{}) { + colMap := make(map[int64]*types.FieldType, 3) + for _, col := range tableInfo.Columns { + colMap[col.ID] = &col.FieldType + } + + if respValue.Info != nil { + var err error + datas := make(map[string]map[string]string) + for _, w := range respValue.Info.Writes { + if len(w.ShortValue) > 0 { + datas[strconv.FormatUint(w.StartTs, 10)], err = decodeMvccRecordValue(w.ShortValue, colMap, tableInfo) + } + } + + for _, v := range respValue.Info.Values { + if len(v.Value) > 0 { + datas[strconv.FormatUint(v.StartTs, 10)], err = decodeMvccRecordValue(v.Value, colMap, tableInfo) + } + } + if len(datas) > 0 { + outMap["decoded"] = datas + if err != nil { + outMap["decode_error"] = err.Error() + } + } + } + } +} + +// DecodeIndexMvccData creates a closure that captures the indexInfo to be used a decode function in GetMvccByKey. +func DecodeIndexMvccData(indexInfo *model.IndexInfo) func(kv.Key, *kvrpcpb.MvccGetByKeyResponse, map[string]interface{}) { + return func(key kv.Key, respValue *kvrpcpb.MvccGetByKeyResponse, outMap map[string]interface{}) { + if respValue.Info != nil { + var ( + hd kv.Handle + err error + datas = make(map[string]map[string]string) + ) + for _, w := range respValue.Info.Writes { + if len(w.ShortValue) > 0 { + hd, err = tablecodec.DecodeIndexHandle(key, w.ShortValue, len(indexInfo.Columns)) + if err == nil { + datas[strconv.FormatUint(w.StartTs, 10)] = map[string]string{"handle": hd.String()} + } + } + } + for _, v := range respValue.Info.Values { + if len(v.Value) > 0 { + hd, err = tablecodec.DecodeIndexHandle(key, v.Value, len(indexInfo.Columns)) + if err == nil { + datas[strconv.FormatUint(v.StartTs, 10)] = map[string]string{"handle": hd.String()} + } + } + } + if len(datas) > 0 { + outMap["decoded"] = datas + if err != nil { + outMap["decode_error"] = err.Error() + } + } + } + } +} + +func decodeMvccRecordValue(bs []byte, colMap map[int64]*types.FieldType, tb *model.TableInfo) (map[string]string, error) { + rs, err := tablecodec.DecodeRowToDatumMap(bs, colMap, time.UTC) + record := make(map[string]string, len(tb.Columns)) + for _, col := range tb.Columns { + if c, ok := rs[col.ID]; ok { + data := "nil" + if !c.IsNull() { + data, err = c.ToString() + } + record[col.Name.O] = data + } + } + return record, err +} + +// ReportLookupInconsistent reports inconsistent when index rows is more than record rows. +func (r *Reporter) ReportLookupInconsistent(ctx context.Context, idxCnt, tblCnt int, missHd, fullHd []kv.Handle, missRowIdx []RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("indexLookup found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Int("index_cnt", idxCnt), + zap.Int("table_cnt", tblCnt), + zap.Stack("stack")) + } else { + const maxFullHandleCnt = 50 + displayFullHdCnt := len(fullHd) + if displayFullHdCnt > maxFullHandleCnt { + displayFullHdCnt = maxFullHandleCnt + } + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Int("index_cnt", idxCnt), zap.Int("table_cnt", tblCnt), + zap.String("missing_handles", fmt.Sprint(missHd)), + zap.String("total_handles", fmt.Sprint(fullHd[:displayFullHdCnt])), + } + store, ok := r.Sctx.GetStore().(helper.Storage) + if ok { + for i, hd := range missHd { + fs = append(fs, zap.String("row_mvcc_"+strconv.Itoa(i), GetMvccByKey(store, r.HandleEncode(hd), DecodeRowMvccData(r.Tbl)))) + } + for i := range missRowIdx { + fs = append(fs, zap.String("index_mvcc_"+strconv.Itoa(i), GetMvccByKey(store, r.IndexEncode(&missRowIdx[i]), DecodeIndexMvccData(r.Idx)))) + } + } + + logutil.Logger(ctx).Error("indexLookup found data inconsistency", fs...) + } + return ErrLookupInconsistent.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, idxCnt, tblCnt) +} + +// ReportAdminCheckInconsistentWithColInfo reports inconsistent when the value of index row is different from record row. +func (r *Reporter) ReportAdminCheckInconsistentWithColInfo(ctx context.Context, handle kv.Handle, colName string, idxDat, tblDat fmt.Stringer, err error, idxRow *RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("admin check found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index", r.Idx.Name.O), + zap.String("col", colName), + zap.Error(err), + zap.Stack("stack"), + ) + } else { + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.String("col", colName), + zap.Stringer("row_id", handle), + zap.Stringer("idxDatum", idxDat), + zap.Stringer("rowDatum", tblDat), + } + store, ok := r.Sctx.GetStore().(helper.Storage) + if ok { + fs = append(fs, zap.String("row_mvcc", GetMvccByKey(store, r.HandleEncode(handle), DecodeRowMvccData(r.Tbl)))) + fs = append(fs, zap.String("index_mvcc", GetMvccByKey(store, r.IndexEncode(idxRow), DecodeIndexMvccData(r.Idx)))) + } + fs = append(fs, zap.Error(err)) + fs = append(fs, zap.Stack("stack")) + logutil.Logger(ctx).Error("admin check found data inconsistency", fs...) + } + return ErrAdminCheckInconsistentWithColInfo.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, colName, fmt.Sprint(handle), fmt.Sprint(idxDat), fmt.Sprint(tblDat), err) +} + +// RecordData is the record data composed of a handle and values. +type RecordData struct { + Handle kv.Handle + Values []types.Datum +} + +func (r *RecordData) String() string { + if r == nil { + return "" + } + return fmt.Sprintf("handle: %s, values: %s", fmt.Sprint(r.Handle), fmt.Sprint(r.Values)) +} + +// ReportAdminCheckInconsistent reports inconsistent when single index row not found in record rows. +func (r *Reporter) ReportAdminCheckInconsistent(ctx context.Context, handle kv.Handle, idxRow, tblRow *RecordData) error { + if r.Sctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("admin check found data inconsistency", + zap.String("table_name", r.Tbl.Name.O), + zap.String("index", r.Idx.Name.O), + zap.Stack("stack"), + ) + } else { + fs := []zap.Field{ + zap.String("table_name", r.Tbl.Name.O), + zap.String("index_name", r.Idx.Name.O), + zap.Stringer("row_id", handle), + zap.Stringer("index", idxRow), + zap.Stringer("row", tblRow), + } + store, ok := r.Sctx.GetStore().(helper.Storage) + if ok { + fs = append(fs, zap.String("row_mvcc", GetMvccByKey(store, r.HandleEncode(handle), DecodeRowMvccData(r.Tbl)))) + if idxRow != nil { + fs = append(fs, zap.String("index_mvcc", GetMvccByKey(store, r.IndexEncode(idxRow), DecodeIndexMvccData(r.Idx)))) + } + } + fs = append(fs, zap.Stack("stack")) + logutil.Logger(ctx).Error("admin check found data inconsistency", fs...) + } + return ErrAdminCheckInconsistent.GenWithStackByArgs(r.Tbl.Name.O, r.Idx.Name.O, fmt.Sprint(handle), fmt.Sprint(idxRow), fmt.Sprint(tblRow)) +} diff --git a/util/logutil/log.go b/util/logutil/log.go index a9e7efa6a0b78..e56245d4be9f0 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -179,12 +179,14 @@ func SetLevel(level string) error { type ctxLogKeyType struct{} -var ctxLogKey = ctxLogKeyType{} +// CtxLogKey indicates the context key for logger +// public for test usage. +var CtxLogKey = ctxLogKeyType{} // Logger gets a contextual logger from current context. // contextual logger will output common fields from context. func Logger(ctx context.Context) *zap.Logger { - if ctxlogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxlogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { return ctxlogger } return log.L() @@ -198,23 +200,23 @@ func BgLogger() *zap.Logger { // WithConnID attaches connId to context. func WithConnID(ctx context.Context, connID uint64) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, logger.With(zap.Uint64("conn", connID))) + return context.WithValue(ctx, CtxLogKey, logger.With(zap.Uint64("conn", connID))) } // WithTraceLogger attaches trace identifier to context func WithTraceLogger(ctx context.Context, connID uint64) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, wrapTraceLogger(ctx, connID, logger)) + return context.WithValue(ctx, CtxLogKey, wrapTraceLogger(ctx, connID, logger)) } func wrapTraceLogger(ctx context.Context, connID uint64, logger *zap.Logger) *zap.Logger { @@ -246,12 +248,12 @@ func (t *traceLog) Sync() error { // WithKeyValue attaches key/value to context. func WithKeyValue(ctx context.Context, key, value string) context.Context { var logger *zap.Logger - if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { + if ctxLogger, ok := ctx.Value(CtxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { logger = log.L() } - return context.WithValue(ctx, ctxLogKey, logger.With(zap.String(key, value))) + return context.WithValue(ctx, CtxLogKey, logger.With(zap.String(key, value))) } // TraceEventKey presents the TraceEventKey in span log. diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index 69eb731dee8c6..d059204973678 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -106,12 +106,12 @@ func TestGrpcLoggerCreation(t *testing.T) { _, p, err := initGRPCLogger(conf) // assert after init grpc logger, the original conf is not changed require.Equal(t, conf.Level, level) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, p.Level.Level(), zap.ErrorLevel) os.Setenv("GRPC_DEBUG", "1") defer os.Unsetenv("GRPC_DEBUG") _, newP, err := initGRPCLogger(conf) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, newP.Level.Level(), zap.DebugLevel) } @@ -121,7 +121,7 @@ func TestSlowQueryLoggerCreation(t *testing.T) { _, prop, err := newSlowQueryLogger(conf) // assert after init slow query logger, the original conf is not changed require.Equal(t, conf.Level, level) - require.Nil(t, err) + require.NoError(t, err) // slow query logger doesn't use the level of the global log config, and the // level should be less than WarnLevel which is used by it to log slow query. require.NotEqual(t, conf.Level, prop.Level.String()) diff --git a/util/misc_test.go b/util/misc_test.go index 11b169669816a..5f21efa4a77dc 100644 --- a/util/misc_test.go +++ b/util/misc_test.go @@ -178,8 +178,8 @@ func TestToPB(t *testing.T) { } column2.Collate = "utf8mb4_bin" - assert.Equal(t, "column_id:1 collation:45 columnLen:-1 decimal:-1 ", ColumnToProto(column).String()) - assert.Equal(t, "column_id:1 collation:45 columnLen:-1 decimal:-1 ", ColumnsToProto([]*model.ColumnInfo{column, column2}, false)[0].String()) + assert.Equal(t, "column_id:1 collation:-45 columnLen:-1 decimal:-1 ", ColumnToProto(column).String()) + assert.Equal(t, "column_id:1 collation:-45 columnLen:-1 decimal:-1 ", ColumnsToProto([]*model.ColumnInfo{column, column2}, false)[0].String()) } func TestComposeURL(t *testing.T) { diff --git a/util/parser/parser_test.go b/util/parser/parser_test.go index 4a05fd0146179..d877fbc3a8410 100644 --- a/util/parser/parser_test.go +++ b/util/parser/parser_test.go @@ -34,7 +34,7 @@ func TestSpace(t *testing.T) { } for _, test := range okTable { rest, err := utilparser.Space(test.Input, test.Times) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, test.Expected, rest) } @@ -70,7 +70,7 @@ func TestDigit(t *testing.T) { for _, test := range okTable { digits, rest, err := utilparser.Digit(test.Input, test.Times) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, test.ExpectedDigits, digits) require.Equal(t, test.ExpectedRest, rest) } @@ -107,7 +107,7 @@ func TestNumber(t *testing.T) { for _, test := range okTable { digits, rest, err := utilparser.Number(test.Input) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, test.ExpectedNum, digits) require.Equal(t, test.ExpectedRest, rest) } @@ -143,12 +143,12 @@ func TestCharAndAnyChar(t *testing.T) { for _, test := range okTable { rest, err := utilparser.Char(test.Input, test.Char) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, test.Expected, rest) rest, err = utilparser.AnyChar(test.Input) - require.Nil(t, err) + require.NoError(t, err) require.Equal(t, test.Expected, rest) } diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 0ce75f5d5a6a0..ca315c1544d7e 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -16,12 +16,13 @@ package pdapi // The following constants are the APIs of PD server. const ( - HotRead = "/pd/api/v1/hotspot/regions/read" - HotWrite = "/pd/api/v1/hotspot/regions/write" - HotHistory = "/pd/api/v1/hotspot/regions/history" - Regions = "/pd/api/v1/regions" - RegionByID = "/pd/api/v1/region/id/" - Stores = "/pd/api/v1/stores" - Status = "/pd/api/v1/status" - Config = "/pd/api/v1/config" + HotRead = "/pd/api/v1/hotspot/regions/read" + HotWrite = "/pd/api/v1/hotspot/regions/write" + HotHistory = "/pd/api/v1/hotspot/regions/history" + Regions = "/pd/api/v1/regions" + RegionByID = "/pd/api/v1/region/id" + StoreRegions = "/pd/api/v1/regions/store" + Stores = "/pd/api/v1/stores" + Status = "/pd/api/v1/status" + Config = "/pd/api/v1/config" ) diff --git a/util/prefix_helper_test.go b/util/prefix_helper_test.go index 94e04add5669d..4f44d487b019c 100644 --- a/util/prefix_helper_test.go +++ b/util/prefix_helper_test.go @@ -35,42 +35,42 @@ const ( func TestPrefix(t *testing.T) { s, err := mockstore.NewMockStore() - require.Nil(t, err) + require.NoError(t, err) defer func() { err := s.Close() - require.Nil(t, err) + require.NoError(t, err) }() ctx := &mockContext{10000000, make(map[fmt.Stringer]interface{}), s, nil} err = ctx.fillTxn() - require.Nil(t, err) + require.NoError(t, err) txn, err := ctx.GetTxn() - require.Nil(t, err) + require.NoError(t, err) err = util.DelKeyWithPrefix(txn, encodeInt(ctx.prefix)) - require.Nil(t, err) + require.NoError(t, err) err = ctx.CommitTxn() - require.Nil(t, err) + require.NoError(t, err) txn, err = s.Begin() - require.Nil(t, err) + require.NoError(t, err) k := []byte("key100jfowi878230") err = txn.Set(k, []byte(`val32dfaskli384757^*&%^`)) - require.Nil(t, err) + require.NoError(t, err) err = util.ScanMetaWithPrefix(txn, k, func(kv.Key, []byte) bool { return true }) - require.Nil(t, err) + require.NoError(t, err) err = util.ScanMetaWithPrefix(txn, k, func(kv.Key, []byte) bool { return false }) - require.Nil(t, err) + require.NoError(t, err) err = util.DelKeyWithPrefix(txn, []byte("key")) - require.Nil(t, err) + require.NoError(t, err) _, err = txn.Get(context.TODO(), k) assert.True(t, terror.ErrorEqual(kv.ErrNotExist, err)) err = txn.Commit(context.Background()) - require.Nil(t, err) + require.NoError(t, err) } func TestPrefixFilter(t *testing.T) { diff --git a/util/profile/flamegraph_test.go b/util/profile/flamegraph_test.go index f26c45589be46..aa0a36ca365d3 100644 --- a/util/profile/flamegraph_test.go +++ b/util/profile/flamegraph_test.go @@ -27,14 +27,14 @@ import ( func TestProfileToDatum(t *testing.T) { file, err := os.Open("testdata/test.pprof") - require.Nil(t, err) + require.NoError(t, err) defer func() { err := file.Close() - require.Nil(t, err) + require.NoError(t, err) }() data, err := (&Collector{}).ProfileReaderToDatums(file) - require.Nil(t, err) + require.NoError(t, err) datums := [][]types.Datum{ types.MakeDatums(`root`, "100%", "100%", 0, 0, `root`), diff --git a/util/profile/main_test.go b/util/profile/main_test.go index d3ddfb5ac3238..90a8953ef4974 100644 --- a/util/profile/main_test.go +++ b/util/profile/main_test.go @@ -24,7 +24,7 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } goleak.VerifyTestMain(m, opts...) diff --git a/util/profile/profile_test.go b/util/profile/profile_test.go index e9fbfa0c776a4..0e6ba8d4f6c1d 100644 --- a/util/profile/profile_test.go +++ b/util/profile/profile_test.go @@ -33,15 +33,15 @@ func TestProfiles(t *testing.T) { var dom *domain.Domain store, err = mockstore.NewMockStore() - require.Nil(t, err) + require.NoError(t, err) defer func() { err := store.Close() - require.Nil(t, err) + require.NoError(t, err) }() session.DisableStats4Test() dom, err = session.BootstrapSession(store) - require.Nil(t, err) + require.NoError(t, err) defer dom.Close() oldValue := profile.CPUProfileInterval diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index ca4264397746e..ef85fde6fb738 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -197,7 +198,7 @@ func extractIndexPointRangesForCNF(sctx sessionctx.Context, conds []expression.E offset := int(-1) for i, cond := range conds { tmpConds := []expression.Expression{cond} - colSets := expression.ExtractColumnSet(tmpConds) + colSets := expression.ExtractColumnSet(cond) if colSets.Len() == 0 { continue } @@ -877,3 +878,386 @@ func MergeDNFItems4Col(ctx sessionctx.Context, dnfItems []expression.Expression) } return mergedDNFItems } + +// AddGcColumnCond add the `tidb_shard(x) = xxx` to the condition +// @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] +// @param[in] accessCond the conditions relative to the index and arranged by the index column order. +// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is +// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is +// only relative to uk's columns. +// @param[in] columnValues the values of index columns in param accessCond. if accessCond is {a = 2, b = 1}, +// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues +// is empty. +// @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix +// error if error gernerated, return error +func AddGcColumnCond(sctx sessionctx.Context, + cols []*expression.Column, + accessesCond []expression.Expression, + columnValues []*valueInfo) ([]expression.Expression, error) { + + if cond := accessesCond[1]; cond != nil { + if f, ok := cond.(*expression.ScalarFunction); ok { + switch f.FuncName.L { + case ast.EQ: + return AddGcColumn4EqCond(sctx, cols, accessesCond, columnValues) + case ast.In: + return AddGcColumn4InCond(sctx, cols, accessesCond) + } + } + } + + return accessesCond, nil +} + +// AddGcColumn4InCond add the `tidb_shard(x) = xxx` for `IN` condition +// For param explanation, please refer to the function `AddGcColumnCond`. +// @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix +// error if error gernerated, return error +func AddGcColumn4InCond(sctx sessionctx.Context, + cols []*expression.Column, + accessesCond []expression.Expression) ([]expression.Expression, error) { + + var errRes error + var newAccessCond []expression.Expression + record := make([]types.Datum, 1) + + expr := cols[0].VirtualExpr.Clone() + andType := types.NewFieldType(mysql.TypeTiny) + + sf := accessesCond[1].(*expression.ScalarFunction) + c := sf.GetArgs()[0].(*expression.Column) + var andOrExpr expression.Expression + for i, arg := range sf.GetArgs()[1:] { + // get every const value and calculate tidb_shard(val) + con := arg.(*expression.Constant) + conVal, err := con.Eval(chunk.Row{}) + if err != nil { + return accessesCond, err + } + + record[0] = conVal + mutRow := chunk.MutRowFromDatums(record) + exprVal, err := expr.Eval(mutRow.ToRow()) + if err != nil { + return accessesCond, err + } + + // tmpArg1 is like `tidb_shard(a) = 8`, tmpArg2 is like `a = 100` + exprCon := &expression.Constant{Value: exprVal, RetType: cols[0].RetType} + tmpArg1, err := expression.NewFunction(sctx, ast.EQ, cols[0].RetType, cols[0], exprCon) + if err != nil { + return accessesCond, err + } + tmpArg2, err := expression.NewFunction(sctx, ast.EQ, c.RetType, c.Clone(), arg) + if err != nil { + return accessesCond, err + } + + // make a LogicAnd, e.g. `tidb_shard(a) = 8 AND a = 100` + andExpr, err := expression.NewFunction(sctx, ast.LogicAnd, andType, tmpArg1, tmpArg2) + if err != nil { + return accessesCond, err + } + + if i == 0 { + andOrExpr = andExpr + } else { + // if the LogicAnd more than one, make a LogicOr, + // e.g. `(tidb_shard(a) = 8 AND a = 100) OR (tidb_shard(a) = 161 AND a = 200)` + andOrExpr, errRes = expression.NewFunction(sctx, ast.LogicOr, andType, andOrExpr, andExpr) + if errRes != nil { + return accessesCond, errRes + } + } + } + + newAccessCond = append(newAccessCond, andOrExpr) + + return newAccessCond, nil +} + +// AddGcColumn4EqCond add the `tidb_shard(x) = xxx` prefix for equal condition +// For param explanation, please refer to the function `AddGcColumnCond`. +// @retval - []expression.Expression the new conditions after adding `tidb_shard() = xxx` prefix +// []*valueInfo the values of every columns in the returned new conditions +// error if error gernerated, return error +func AddGcColumn4EqCond(sctx sessionctx.Context, + cols []*expression.Column, + accessesCond []expression.Expression, + columnValues []*valueInfo) ([]expression.Expression, error) { + + expr := cols[0].VirtualExpr.Clone() + record := make([]types.Datum, len(columnValues)-1) + + for i := 1; i < len(columnValues); i++ { + cv := columnValues[i] + if cv == nil { + break + } + record[i-1] = *cv.value + } + + mutRow := chunk.MutRowFromDatums(record) + evaluated, err := expr.Eval(mutRow.ToRow()) + if err != nil { + return accessesCond, err + } + vi := &valueInfo{false, &evaluated} + con := &expression.Constant{Value: evaluated, RetType: cols[0].RetType} + // make a tidb_shard() function, e.g. `tidb_shard(a) = 8` + cond, err := expression.NewFunction(sctx, ast.EQ, cols[0].RetType, cols[0], con) + if err != nil { + return accessesCond, err + } + + accessesCond[0] = cond + columnValues[0] = vi + return accessesCond, nil +} + +// AddExpr4EqAndInCondition add the `tidb_shard(x) = xxx` prefix +// Add tidb_shard() for EQ and IN function. e.g. input condition is `WHERE a = 1`, +// output condition is `WHERE tidb_shard(a) = 214 AND a = 1`. e.g. input condition +// is `WHERE a IN (1, 2 ,3)`, output condition is `WHERE (tidb_shard(a) = 214 AND a = 1) +// OR (tidb_shard(a) = 143 AND a = 2) OR (tidb_shard(a) = 156 AND a = 3)` +// @param[in] conditions the original condition to be processed +// @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] +// @param[in] lengths the length for every column of shard index +// @retval - the new condition after adding tidb_shard() prefix +func AddExpr4EqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, + cols []*expression.Column) ([]expression.Expression, error) { + + accesses := make([]expression.Expression, len(cols)) + columnValues := make([]*valueInfo, len(cols)) + offsets := make([]int, len(conditions)) + addGcCond := true + + // the array accesses stores conditions of every column in the index in the definition order + // e.g. the original condition is `WHERE b = 100 AND a = 200 AND c = 300`, the definition of + // index is (tidb_shard(a), a, b), then accesses is "[a = 200, b = 100]" + for i, cond := range conditions { + offset := getPotentialEqOrInColOffset(sctx, cond, cols) + offsets[i] = offset + if offset == -1 { + continue + } + if accesses[offset] == nil { + accesses[offset] = cond + continue + } + // if the same field appear twice or more, don't add tidb_shard() + // e.g. `WHERE a > 100 and a < 200` + addGcCond = false + } + + for i, cond := range accesses { + if cond == nil { + continue + } + if !allEqOrIn(cond) { + addGcCond = false + break + } + columnValues[i] = extractValueInfo(cond) + } + + if !addGcCond || !NeedAddGcColumn4ShardIndex(cols, accesses, columnValues) { + return conditions, nil + } + + // remove the accesses from newConditions + newConditions := make([]expression.Expression, 0, len(conditions)) + newConditions = append(newConditions, conditions...) + newConditions = removeAccessConditions(newConditions, accesses) + + // add Gc condition for accesses and return new condition to newAccesses + newAccesses, err := AddGcColumnCond(sctx, cols, accesses, columnValues) + if err != nil { + return conditions, err + } + + // merge newAccesses and original condition execept accesses + newConditions = append(newConditions, newAccesses...) + + return newConditions, nil +} + +// NeedAddGcColumn4ShardIndex check whether to add `tidb_shard(x) = xxx` +// @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] +// @param[in] accessCond the conditions relative to the index and arranged by the index column order. +// e.g. the index is uk(tidb_shard(a), a, b) and the where clause is +// `WHERE b = 1 AND a = 2 AND c = 3`, the param accessCond is {a = 2, b = 1} that is +// only relative to uk's columns. +// @param[in] columnValues the values of index columns in param accessCond. if accessCond is {a = 2, b = 1}, +// columnValues is {2, 1}. if accessCond the "IN" function like `a IN (1, 2)`, columnValues +// is empty. +// @retval - return true if it needs to addr tidb_shard() prefix, ohterwise return false +func NeedAddGcColumn4ShardIndex( + cols []*expression.Column, + accessCond []expression.Expression, + columnValues []*valueInfo) bool { + + // the columns of shard index shoude be more than 2, like (tidb_shard(a),a,...) + // check cols and columnValues in the sub call function + if len(accessCond) < 2 || len(cols) < 2 { + return false + } + + if !IsValidShardIndex(cols) { + return false + } + + // accessCond[0] shoudle be nil, because it has no access condition for + // the prefix tidb_shard() of the shard index + if cond := accessCond[1]; cond != nil { + if f, ok := cond.(*expression.ScalarFunction); ok { + switch f.FuncName.L { + case ast.EQ: + return NeedAddColumn4EqCond(cols, accessCond, columnValues) + case ast.In: + return NeedAddColumn4InCond(cols, accessCond, f) + } + } + } + + return false +} + +// NeedAddColumn4EqCond `tidb_shard(x) = xxx` +// For param explanation, please refer to the function `NeedAddGcColumn4ShardIndex`. +// It checks whether EQ conditions need to be added tidb_shard() prefix. +// (1) columns in accessCond are all columns of the index except the first. +// (2) every column in accessCond has a constan value +func NeedAddColumn4EqCond(cols []*expression.Column, + accessCond []expression.Expression, columnValues []*valueInfo) bool { + valCnt := 0 + matchedKeyFldCnt := 0 + + // the columns of shard index shoude be more than 2, like (tidb_shard(a),a,...) + if len(columnValues) < 2 { + return false + } + + for _, cond := range accessCond[1:] { + if cond == nil { + break + } + + f, ok := cond.(*expression.ScalarFunction) + if !ok || f.FuncName.L != ast.EQ { + return false + } + + matchedKeyFldCnt++ + } + for _, val := range columnValues[1:] { + if val == nil { + break + } + valCnt++ + } + + if matchedKeyFldCnt != len(cols)-1 || + valCnt != len(cols)-1 || + accessCond[0] != nil || + columnValues[0] != nil { + return false + } + + return true +} + +// NeedAddColumn4InCond `tidb_shard(x) = xxx` +// For param explanation, please refer to the function `NeedAddGcColumn4ShardIndex`. +// It checks whether "IN" conditions need to be added tidb_shard() prefix. +// (1) columns in accessCond are all columns of the index except the first. +// (2) the first param of "IN" function should be a column not a expression like `a + b` +// (3) the rest params of "IN" function all should be constant +// (4) the first param of "IN" function should be the column in the expression of first index field. +// e.g. uk(tidb_shard(a), a). If the conditions is `WHERE b in (1, 2, 3)`, the first param of "IN" function +// is `b` that's not the column in `tidb_shard(a)`. +// @param sf "IN" function, e.g. `a IN (1, 2, 3)` +func NeedAddColumn4InCond(cols []*expression.Column, accessCond []expression.Expression, sf *expression.ScalarFunction) bool { + if len(cols) == 0 || len(accessCond) == 0 || sf == nil { + return false + } + + if accessCond[0] != nil { + return false + } + + fields := ExtractColumnsFromExpr(cols[0].VirtualExpr.(*expression.ScalarFunction)) + + c, ok := sf.GetArgs()[0].(*expression.Column) + if !ok { + return false + } + + for _, arg := range sf.GetArgs()[1:] { + if _, ok := arg.(*expression.Constant); !ok { + return false + } + } + + if len(fields) != 1 || + !fields[0].Equal(nil, c) { + return false + } + + return true +} + +// ExtractColumnsFromExpr get all fields from input expression virtaulExpr +func ExtractColumnsFromExpr(virtaulExpr *expression.ScalarFunction) []*expression.Column { + var fields []*expression.Column + + if virtaulExpr == nil { + return fields + } + + for _, arg := range virtaulExpr.GetArgs() { + if sf, ok := arg.(*expression.ScalarFunction); ok { + fields = append(fields, ExtractColumnsFromExpr(sf)...) + } else if c, ok := arg.(*expression.Column); ok { + if !c.InColumnArray(fields) { + fields = append(fields, c) + } + } + } + + return fields +} + +// IsValidShardIndex Check whether the definition of shard index is valid. The form of index +// should like `index(tidb_shard(a), a, ....)`. +// 1) the column count shoudle be >= 2 +// 2) the first column should be tidb_shard(xxx) +// 3) the parameter of tidb_shard shoudle be a column that is the second column of index +// @param[in] cols the columns of shard index, such as [tidb_shard(a), a, ...] +// @retval - if the shard index is valid return true, otherwise return false +func IsValidShardIndex(cols []*expression.Column) bool { + // definition of index should like the form: index(tidb_shard(a), a, ....) + if len(cols) < 2 { + return false + } + + // the first coulmn of index must be GC column and the expr must be tidb_shard + if !expression.GcColumnExprIsTidbShard(cols[0].VirtualExpr) { + return false + } + + shardFunc, _ := cols[0].VirtualExpr.(*expression.ScalarFunction) + + argCount := len(shardFunc.GetArgs()) + if argCount != 1 { + return false + } + + // parameter of tidb_shard must be the second column of the input index columns + col, ok := shardFunc.GetArgs()[0].(*expression.Column) + if !ok || !col.Equal(nil, cols[1]) { + return false + } + + return true +} diff --git a/util/ranger/main_test.go b/util/ranger/main_test.go index 410ac374b04d6..015e587bd0e74 100644 --- a/util/ranger/main_test.go +++ b/util/ranger/main_test.go @@ -43,7 +43,7 @@ func TestMain(m *testing.M) { testDataMap.GenerateOutputIfNeeded() opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } diff --git a/util/ranger/points.go b/util/ranger/points.go index 3633c15690634..8b2fcf52fdf0a 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -458,6 +458,10 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val res = append(res, &point{value: d, excl: false, start: false}) } + if op == ast.NullEQ && val.IsNull() { + res = append(res, &point{start: true}, &point{}) // null point + } + tmpEnum := types.Enum{} for i := 0; i <= len(ft.Elems); i++ { if i == 0 { @@ -486,7 +490,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val if v >= 0 { appendPointFunc(d) } - case ast.EQ: + case ast.EQ, ast.NullEQ: if v == 0 { appendPointFunc(d) } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 3e9270f268509..9a35b94140a80 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -19,42 +19,25 @@ import ( "fmt" "testing" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/mysql" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/testkit/testdata" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" ) -func newDomainStoreWithBootstrap(t *testing.T) (*domain.Domain, kv.Storage, error) { - store, err := mockstore.NewMockStore() - require.NoError(t, err) - session.SetSchemaLease(0) - session.DisableStats4Test() - if err != nil { - return nil, nil, errors.Trace(err) - } - dom, err := session.BootstrapSession(store) - return dom, store, errors.Trace(err) -} - func TestTableRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -307,12 +290,9 @@ func TestTableRange(t *testing.T) { // for issue #6661 func TestIndexRangeForUnsignedAndOverflow(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -498,12 +478,9 @@ create table t( } func TestColumnRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -863,12 +840,9 @@ func TestColumnRange(t *testing.T) { } func TestIndexRangeEliminatedProjection(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -890,12 +864,9 @@ func TestIndexRangeEliminatedProjection(t *testing.T) { } func TestCompIndexInExprCorrCol(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -919,12 +890,9 @@ func TestCompIndexInExprCorrCol(t *testing.T) { } func TestIndexStringIsTrueRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t0") @@ -949,12 +917,9 @@ func TestIndexStringIsTrueRange(t *testing.T) { } func TestCompIndexDNFMatch(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec(`set @@session.tidb_regard_null_as_point=false`) @@ -981,12 +946,9 @@ func TestCompIndexDNFMatch(t *testing.T) { } func TestCompIndexMultiColDNF1(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn @@ -1014,12 +976,9 @@ func TestCompIndexMultiColDNF1(t *testing.T) { } func TestCompIndexMultiColDNF2(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn @@ -1047,12 +1006,9 @@ func TestCompIndexMultiColDNF2(t *testing.T) { } func TestPrefixIndexMultiColDNF(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test;") testKit.MustExec("drop table if exists t2;") @@ -1082,12 +1038,9 @@ func TestPrefixIndexMultiColDNF(t *testing.T) { } func TestIndexRangeForBit(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test;") testKit.MustExec("set @@tidb_partition_prune_mode = 'static';") @@ -1120,12 +1073,9 @@ func TestIndexRangeForBit(t *testing.T) { } func TestIndexRangeForYear(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) // for issue #20101: overflow when converting integer to year @@ -1282,12 +1232,9 @@ func TestIndexRangeForYear(t *testing.T) { // For https://github.com/pingcap/tidb/issues/22032 func TestPrefixIndexRangeScan(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") @@ -1353,12 +1300,9 @@ func TestPrefixIndexRangeScan(t *testing.T) { } func TestIndexRangeForDecimal(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test;") testKit.MustExec("drop table if exists t1, t2;") @@ -1386,12 +1330,9 @@ func TestIndexRangeForDecimal(t *testing.T) { } func TestPrefixIndexAppendPointRanges(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() - require.NoError(t, err) + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("USE test") testKit.MustExec("DROP TABLE IF EXISTS IDT_20755") @@ -1423,12 +1364,9 @@ func TestPrefixIndexAppendPointRanges(t *testing.T) { } func TestIndexRange(t *testing.T) { - dom, store, err := newDomainStoreWithBootstrap(t) - require.NoError(t, err) - defer func() { - dom.Close() - require.NoError(t, store.Close()) - }() + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -1741,8 +1679,6 @@ create table t( }, } - collate.SetNewCollationEnabledForTest(true) - defer func() { collate.SetNewCollationEnabledForTest(false) }() ctx := context.Background() for _, tt := range tests { t.Run(tt.exprStr, func(t *testing.T) { @@ -1774,3 +1710,409 @@ create table t( }) } } + +func TestTableShardIndex(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + config.UpdateGlobal(func(conf *config.Config) { + conf.Experimental.AllowsExpressionIndex = true + }) + testKit.MustExec("use test") + testKit.MustExec("drop table if exists test3") + testKit.MustExec("create table test3(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a))") + testKit.MustExec("create table test33(id int primary key clustered, a int, b int, unique key a(a))") + testKit.MustExec("create table test4(id int primary key clustered, a int, b int, " + + "unique key uk_expr((tidb_shard(a)),a),unique key uk_b_expr((tidb_shard(b)),b))") + testKit.MustExec("create table test5(id int primary key clustered, a int, b int, " + + "unique key uk_expr((tidb_shard(a)),a,b))") + testKit.MustExec("create table test6(id int primary key clustered, a int, b int, c int, " + + "unique key uk_expr((tidb_shard(a)), a))") + testKit.MustExec("create table testx(id int primary key clustered, a int, b int, unique key a(a))") + testKit.MustExec("create table testy(id int primary key clustered, a int, b int, " + + "unique key uk_expr((tidb_shard(b)),a))") + testKit.MustExec("create table testz(id int primary key clustered, a int, b int, " + + "unique key uk_expr((tidb_shard(a+b)),a))") + + tests := []struct { + exprStr string + accessConds string + childLevel int + tableName string + }{ + { + exprStr: "a = 1", + accessConds: "[eq(tidb_shard(test.test3.a), 214) eq(test.test3.a, 1)]", + tableName: "test3", + }, + { + exprStr: "a=100 and (b = 100 or b = 200)", + accessConds: "[or(eq(test.test3.b, 100), eq(test.test3.b, 200)) eq(tidb_shard(test.test3.a), 8) " + + "eq(test.test3.a, 100)]", + tableName: "test3", + }, + { + // don't add prefix + exprStr: " tidb_shard(a) = 8", + accessConds: "[eq(tidb_shard(test.test3.a), 8)]", + tableName: "test3", + }, + { + exprStr: "a=100 or b = 200", + accessConds: "[or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), " + + "eq(test.test3.b, 200))]", + tableName: "test3", + }, + { + exprStr: "a=100 or b > 200", + accessConds: "[or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), " + + "gt(test.test3.b, 200))]", + tableName: "test3", + }, + { + exprStr: "a=100 or a = 200 or 1", + accessConds: "[or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), " + + "or(and(eq(tidb_shard(test.test3.a), 161), eq(test.test3.a, 200)), 1))]", + tableName: "test3", + }, + { + exprStr: "(a=100 and b = 100) or a = 300", + accessConds: "[or(and(eq(test.test3.b, 100), and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100))), " + + "and(eq(tidb_shard(test.test3.a), 227), eq(test.test3.a, 300)))]", + tableName: "test3", + }, + { + exprStr: "((a=100 and b = 100) or a = 200) or a = 300", + accessConds: "[or(and(eq(test.test3.b, 100), and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100))), " + + "or(and(eq(tidb_shard(test.test3.a), 161), eq(test.test3.a, 200)), " + + "and(eq(tidb_shard(test.test3.a), 227), eq(test.test3.a, 300))))]", + tableName: "test3", + }, + { + exprStr: "a IN (100, 200, 300)", + accessConds: "[or(or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), " + + "and(eq(tidb_shard(test.test3.a), 161), eq(test.test3.a, 200))), and(eq(tidb_shard(test.test3.a), 227), eq(test.test3.a, 300)))]", + tableName: "test3", + }, + { + exprStr: "a IN (100)", + accessConds: "[eq(tidb_shard(test.test3.a), 8) eq(test.test3.a, 100)]", + tableName: "test3", + }, + { + exprStr: "a IN (100, 200, 300) or a = 400", + accessConds: "[or(or(or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), " + + "and(eq(tidb_shard(test.test3.a), 161), eq(test.test3.a, 200))), and(eq(tidb_shard(test.test3.a), 227), eq(test.test3.a, 300))), and(eq(tidb_shard(test.test3.a), 85), eq(test.test3.a, 400)))]", + tableName: "test3", + }, + { + // don't add prefix + exprStr: "((a=100 and b = 100) or a = 200) and b = 300", + accessConds: "[or(and(eq(test.test3.a, 100), eq(test.test3.b, 100)), eq(test.test3.a, 200)) " + + "eq(test.test3.b, 300)]", + tableName: "test3", + }, + { + // don't add prefix + exprStr: "a = b", + accessConds: "[eq(test.test3.a, test.test3.b)]", + tableName: "test3", + }, + { + // don't add prefix + exprStr: "a = b and b = 100", + accessConds: "[eq(test.test3.a, test.test3.b) eq(test.test3.b, 100)]", + tableName: "test3", + }, + { + // don't add prefix + exprStr: "a > 900", + accessConds: "[gt(test.test3.a, 900)]", + tableName: "test3", + }, + { + // add prefix + exprStr: "a = 3 or a > 900", + accessConds: "[or(and(eq(tidb_shard(test.test3.a), 156), eq(test.test3.a, 3)), gt(test.test3.a, 900))]", + tableName: "test3", + }, + // two shard index in one table + { + exprStr: "a = 100", + accessConds: "[eq(tidb_shard(test.test4.a), 8) eq(test.test4.a, 100)]", + tableName: "test4", + }, + { + exprStr: "b = 100", + accessConds: "[eq(tidb_shard(test.test4.b), 8) eq(test.test4.b, 100)]", + tableName: "test4", + }, + { + exprStr: "a = 100 and b = 100", + accessConds: "[eq(tidb_shard(test.test4.a), 8) eq(test.test4.a, 100) " + + "eq(tidb_shard(test.test4.b), 8) eq(test.test4.b, 100)]", + tableName: "test4", + }, + { + exprStr: "a = 100 or b = 100", + accessConds: "[or(and(eq(tidb_shard(test.test4.a), 8), eq(test.test4.a, 100)), " + + "and(eq(tidb_shard(test.test4.b), 8), eq(test.test4.b, 100)))]", + tableName: "test4", + }, + // shard index cotans three fields + { + exprStr: "a = 100 and b = 100", + accessConds: "[eq(tidb_shard(test.test5.a), 8) eq(test.test5.a, 100) eq(test.test5.b, 100)]", + tableName: "test5", + }, + { + exprStr: "(a=100 and b = 100) or (a=200 and b = 200)", + accessConds: "[or(and(eq(tidb_shard(test.test5.a), 8), and(eq(test.test5.a, 100), eq(test.test5.b, 100))), " + + "and(eq(tidb_shard(test.test5.a), 161), and(eq(test.test5.a, 200), eq(test.test5.b, 200))))]", + tableName: "test5", + }, + { + exprStr: "(a, b) in ((100, 100), (200, 200))", + accessConds: "[or(and(eq(tidb_shard(test.test5.a), 8), and(eq(test.test5.a, 100), eq(test.test5.b, 100))), " + + "and(eq(tidb_shard(test.test5.a), 161), and(eq(test.test5.a, 200), eq(test.test5.b, 200))))]", + tableName: "test5", + }, + { + exprStr: "(a, b) in ((100, 100))", + accessConds: "[eq(tidb_shard(test.test5.a), 8) eq(test.test5.a, 100) eq(test.test5.b, 100)]", + tableName: "test5", + }, + // don't add prefix + { + exprStr: "a=100", + accessConds: "[eq(test.testy.a, 100)]", + tableName: "testy", + }, + // don't add prefix + { + exprStr: "a=100", + accessConds: "[eq(test.testz.a, 100)]", + tableName: "testz", + }, + // test join + { + exprStr: "test3.a = 100", + accessConds: "[eq(tidb_shard(test.test3.a), 8) eq(test.test3.a, 100)]", + childLevel: 4, + tableName: "test3 JOIN test33 ON test3.b = test33.b", + }, + { + exprStr: "test3.a = 100 and test33.a > 10", + accessConds: "[gt(test.test33.a, 10) eq(tidb_shard(test.test3.a), 8) eq(test.test3.a, 100)]", + childLevel: 4, + tableName: "test3 JOIN test33 ON test3.b = test33.b", + }, + { + exprStr: "test3.a = 100 AND test6.a = 10", + accessConds: "[eq(test.test6.a, 10) eq(tidb_shard(test.test3.a), 8) eq(test.test3.a, 100)]", + childLevel: 4, + tableName: "test3 JOIN test6 ON test3.b = test6.b", + }, + { + exprStr: "test3.a = 100 or test6.a = 10", + accessConds: "[or(and(eq(tidb_shard(test.test3.a), 8), eq(test.test3.a, 100)), eq(test.test6.a, 10))]", + childLevel: 4, + tableName: "test3 JOIN test6 ON test3.b = test6.b", + }, + } + + ctx := context.Background() + for _, tt := range tests { + t.Run(tt.exprStr, func(t *testing.T) { + sql := "select * from " + tt.tableName + " where " + tt.exprStr + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) + conds := make([]expression.Expression, len(selection.Conditions)) + for i, cond := range selection.Conditions { + conds[i] = expression.PushDownNot(sctx, cond) + } + ds, ok := selection.Children()[0].(*plannercore.DataSource) + if !ok { + if tt.childLevel == 4 { + ds = selection.Children()[0].Children()[0].Children()[0].(*plannercore.DataSource) + } + } + newConds := ds.AddPrefix4ShardIndexes(ds.SCtx(), conds) + require.Equal(t, tt.accessConds, fmt.Sprintf("%s", newConds)) + }) + } + + // test update statement + t.Run("", func(t *testing.T) { + sql := "update test6 set c = 1000 where a=50 and b = 50" + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection, ok := p.(*plannercore.Update).SelectPlan.(*plannercore.PhysicalSelection) + require.True(t, ok) + _, ok = selection.Children()[0].(*plannercore.PointGetPlan) + require.True(t, ok) + }) + + // test delete statement + t.Run("", func(t *testing.T) { + sql := "delete from test6 where a = 45 and b = 45;" + sctx := testKit.Session().(sessionctx.Context) + stmts, err := session.Parse(sctx, sql) + require.NoError(t, err) + require.Len(t, stmts, 1) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) + require.NoError(t, err) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) + require.NoError(t, err) + selection, ok := p.(*plannercore.Delete).SelectPlan.(*plannercore.PhysicalSelection) + require.True(t, ok) + _, ok = selection.Children()[0].(*plannercore.PointGetPlan) + require.True(t, ok) + }) +} + +func TestShardIndexFuncSuites(t *testing.T) { + + store, clean := testkit.CreateMockStore(t) + defer clean() + testKit := testkit.NewTestKit(t, store) + sctx := testKit.Session().(sessionctx.Context) + + // ------------------------------------------- + // test IsValidShardIndex function + // ------------------------------------------- + longlongType := types.NewFieldType(mysql.TypeLonglong) + col0 := &expression.Column{UniqueID: 0, ID: 0, RetType: longlongType} + col1 := &expression.Column{UniqueID: 1, ID: 1, RetType: longlongType} + // col2 is GC column and VirtualExpr = tidb_shard(col0) + col2 := &expression.Column{UniqueID: 2, ID: 2, RetType: longlongType} + col2.VirtualExpr = expression.NewFunctionInternal(sctx, ast.TiDBShard, col2.RetType, col0) + // col3 is GC column and VirtualExpr = abs(col0) + col3 := &expression.Column{UniqueID: 3, ID: 3, RetType: longlongType} + col3.VirtualExpr = expression.NewFunctionInternal(sctx, ast.Abs, col2.RetType, col0) + col4 := &expression.Column{UniqueID: 4, ID: 4, RetType: longlongType} + + cols := []*expression.Column{col0, col1} + + // input is nil + require.False(t, ranger.IsValidShardIndex(nil)) + // only 1 column + require.False(t, ranger.IsValidShardIndex([]*expression.Column{col2})) + // first col is not expression + require.False(t, ranger.IsValidShardIndex(cols)) + // field in tidb_shard is not the secondary column + require.False(t, ranger.IsValidShardIndex([]*expression.Column{col2, col1})) + // expressioin is abs that is not tidb_shard + require.False(t, ranger.IsValidShardIndex([]*expression.Column{col3, col0})) + // normal case + require.True(t, ranger.IsValidShardIndex([]*expression.Column{col2, col0})) + + // ------------------------------------------- + // test ExtractColumnsFromExpr function + // ------------------------------------------- + // normal case + con1 := &expression.Constant{Value: types.NewDatum(1), RetType: longlongType} + con5 := &expression.Constant{Value: types.NewDatum(5), RetType: longlongType} + exprEq := expression.NewFunctionInternal(sctx, ast.EQ, col0.RetType, col0, con1) + exprIn := expression.NewFunctionInternal(sctx, ast.In, col0.RetType, col0, con1, con5) + require.NotNil(t, exprEq) + require.NotNil(t, exprIn) + // input is nil + require.Equal(t, len(ranger.ExtractColumnsFromExpr(nil)), 0) + // input is column + require.Equal(t, len(ranger.ExtractColumnsFromExpr(exprEq.(*expression.ScalarFunction))), 1) + // (col0 = 1 and col3 > 1) or (col4 < 5 and 5) + exprGt := expression.NewFunctionInternal(sctx, ast.GT, longlongType, col3, con1) + require.NotNil(t, exprGt) + andExpr1 := expression.NewFunctionInternal(sctx, ast.And, longlongType, exprEq, exprGt) + require.NotNil(t, andExpr1) + exprLt := expression.NewFunctionInternal(sctx, ast.LT, longlongType, col4, con5) + andExpr2 := expression.NewFunctionInternal(sctx, ast.And, longlongType, exprLt, con5) + orExpr2 := expression.NewFunctionInternal(sctx, ast.Or, longlongType, andExpr1, andExpr2) + require.Equal(t, len(ranger.ExtractColumnsFromExpr(orExpr2.(*expression.ScalarFunction))), 3) + + // ------------------------------------------- + // test NeedAddColumn4InCond function + // ------------------------------------------- + // normal case + sfIn, ok := exprIn.(*expression.ScalarFunction) + require.True(t, ok) + accessCond := []expression.Expression{nil, exprIn} + shardIndexCols := []*expression.Column{col2, col0} + require.True(t, ranger.NeedAddColumn4InCond(shardIndexCols, accessCond, sfIn)) + + // input nil + require.False(t, ranger.NeedAddColumn4InCond(nil, accessCond, sfIn)) + require.False(t, ranger.NeedAddColumn4InCond(shardIndexCols, nil, sfIn)) + require.False(t, ranger.NeedAddColumn4InCond(shardIndexCols, accessCond, nil)) + + // col1 in (1, 5) + exprIn2 := expression.NewFunctionInternal(sctx, ast.In, col1.RetType, col1, con1, con5) + accessCond[1] = exprIn2 + require.False(t, ranger.NeedAddColumn4InCond(shardIndexCols, accessCond, exprIn2.(*expression.ScalarFunction))) + + // col0 in (1, col1) + exprIn3 := expression.NewFunctionInternal(sctx, ast.In, col0.RetType, col1, con1, col1) + accessCond[1] = exprIn3 + require.False(t, ranger.NeedAddColumn4InCond(shardIndexCols, accessCond, exprIn3.(*expression.ScalarFunction))) + + // ------------------------------------------- + // test NeedAddColumn4EqCond function + // ------------------------------------------- + // ranger.valueInfo is not export by package, we can.t test NeedAddColumn4EqCond + eqAccessCond := []expression.Expression{nil, exprEq} + require.False(t, ranger.NeedAddColumn4EqCond(shardIndexCols, eqAccessCond, nil)) + + // ------------------------------------------- + // test NeedAddGcColumn4ShardIndex function + // ------------------------------------------- + // ranger.valueInfo is not export by package, we can.t test NeedAddGcColumn4ShardIndex + require.False(t, ranger.NeedAddGcColumn4ShardIndex(shardIndexCols, nil, nil)) + + // ------------------------------------------- + // test AddExpr4EqAndInCondition function + // ------------------------------------------- + exprIn4 := expression.NewFunctionInternal(sctx, ast.In, col0.RetType, col0, con1) + test := []struct { + inputConds []expression.Expression + outputConds string + }{ + { + // col0 = 1 => tidb_shard(col0) = 214 and col0 = 1 + inputConds: []expression.Expression{exprEq}, + outputConds: "[eq(Column#2, 214) eq(Column#0, 1)]", + }, + { + // col0 in (1) => cols2 = 214 and col0 = 1 + inputConds: []expression.Expression{exprIn4}, + outputConds: "[and(eq(Column#2, 214), eq(Column#0, 1))]", + }, + { + // col0 in (1, 5) => (cols2 = 214 and col0 = 1) or (cols2 = 122 and col0 = 5) + inputConds: []expression.Expression{exprIn}, + outputConds: "[or(and(eq(Column#2, 214), eq(Column#0, 1)), " + + "and(eq(Column#2, 122), eq(Column#0, 5)))]", + }, + } + + for _, tt := range test { + newConds, _ := ranger.AddExpr4EqAndInCondition(sctx, tt.inputConds, shardIndexCols) + require.Equal(t, fmt.Sprintf("%s", newConds), tt.outputConds) + } +} diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index aa39c6c1037e3..0d32f26690a0a 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -82,7 +81,7 @@ func NewRowDecoder(tbl table.Table, cols []*table.Column, decodeColMap map[int64 } // DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value. -func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { +func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { var err error if rowcodec.IsNewFormat(b) { row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) @@ -114,7 +113,7 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. } rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) } - return rd.EvalRemainedExprColumnMap(ctx, sysLoc, row) + return rd.EvalRemainedExprColumnMap(ctx, row) } // BuildFullDecodeColMap builds a map that contains [columnID -> struct{*table.Column, expression.Expression}] from all columns. @@ -175,7 +174,7 @@ func (rd *RowDecoder) DecodeTheExistedColumnMap(ctx sessionctx.Context, handle k // EvalRemainedExprColumnMap is used by ddl column-type-change first column reorg stage. // It is always called after DecodeTheExistedColumnMap to finish the generated column evaluation. -func (rd *RowDecoder) EvalRemainedExprColumnMap(ctx sessionctx.Context, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { +func (rd *RowDecoder) EvalRemainedExprColumnMap(ctx sessionctx.Context, row map[int64]types.Datum) (map[int64]types.Datum, error) { keys := make([]int, 0, len(rd.colMap)) ids := make(map[int]int, len(rd.colMap)) for k, col := range rd.colMap { @@ -198,18 +197,7 @@ func (rd *RowDecoder) EvalRemainedExprColumnMap(ctx sessionctx.Context, sysLoc * return nil, err } - if val.Kind() == types.KindMysqlTime && sysLoc != time.UTC { - t := val.GetMysqlTime() - if t.Type() == mysql.TypeTimestamp { - err := t.ConvertTimeZone(sysLoc, time.UTC) - if err != nil { - return nil, err - } - val.SetMysqlTime(t) - } - } rd.mutRow.SetValue(col.Col.Offset, val.GetValue()) - row[int64(ids[id])] = val } // return the existed and evaluated column map here. diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index 8d1081f528559..ce5d254a90e06 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -61,7 +61,7 @@ func TestRowDecoder(t *testing.T) { decodeColsMap2[col.ID] = tpExpr if col.GeneratedExprString != "" { expr, err := expression.ParseSimpleExprCastWithTableInfo(ctx, col.GeneratedExprString, tblInfo, &col.FieldType) - require.Nil(t, err) + require.NoError(t, err) tpExpr.GenExpr = expr } decodeColsMap[col.ID] = tpExpr @@ -69,8 +69,6 @@ func TestRowDecoder(t *testing.T) { de := decoder.NewRowDecoder(tbl, tbl.Cols(), decodeColsMap) deWithNoGenCols := decoder.NewRowDecoder(tbl, tbl.Cols(), decodeColsMap2) - timeZoneIn8, err := time.LoadLocation("Asia/Shanghai") - require.Nil(t, err) time1 := types.NewTime(types.FromDate(2019, 01, 01, 8, 01, 01, 0), mysql.TypeTimestamp, types.DefaultFsp) t1 := types.NewTimeDatum(time1) d1 := types.NewDurationDatum(types.Duration{ @@ -79,14 +77,10 @@ func TestRowDecoder(t *testing.T) { time2, err := time1.Add(sc, d1.GetMysqlDuration()) require.Nil(t, err) - err = time2.ConvertTimeZone(timeZoneIn8, time.UTC) - require.Nil(t, err) t2 := types.NewTimeDatum(time2) time3, err := time1.Add(sc, types.Duration{Duration: time.Hour*2 + time.Second*2}) require.Nil(t, err) - err = time3.ConvertTimeZone(timeZoneIn8, time.UTC) - require.Nil(t, err) t3 := types.NewTimeDatum(time3) testRows := []struct { @@ -117,10 +111,10 @@ func TestRowDecoder(t *testing.T) { c7.Flag |= mysql.UnsignedFlag } bs, err := tablecodec.EncodeRow(sc, row.input, row.cols, nil, nil, &rd) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, bs) - r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) + r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, nil) require.Nil(t, err) // Last column is primary-key column, and the table primary-key is handle, then the primary-key value won't be // stored in raw data, but store in the raw key. @@ -138,7 +132,7 @@ func TestRowDecoder(t *testing.T) { } } // test decode with no generated column. - r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) + r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, nil) require.Nil(t, err) for k, v := range r2 { v1, ok := r[k] @@ -177,9 +171,6 @@ func TestClusterIndexRowDecoder(t *testing.T) { } de := decoder.NewRowDecoder(tbl, tbl.Cols(), decodeColsMap) - timeZoneIn8, err := time.LoadLocation("Asia/Shanghai") - require.Nil(t, err) - testRows := []struct { cols []int64 input []types.Datum @@ -194,10 +185,10 @@ func TestClusterIndexRowDecoder(t *testing.T) { rd := rowcodec.Encoder{Enable: true} for _, row := range testRows { bs, err := tablecodec.EncodeRow(sc, row.input, row.cols, nil, nil, &rd) - require.Nil(t, err) + require.NoError(t, err) require.NotNil(t, bs) - r, err := de.DecodeAndEvalRowWithMap(ctx, testkit.MustNewCommonHandle(t, 100, "abc"), bs, time.UTC, timeZoneIn8, nil) + r, err := de.DecodeAndEvalRowWithMap(ctx, testkit.MustNewCommonHandle(t, 100, "abc"), bs, time.UTC, nil) require.Nil(t, err) for i, col := range cols { diff --git a/util/rowDecoder/main_test.go b/util/rowDecoder/main_test.go index 1a843cbfe6b69..0fb38e51d6855 100644 --- a/util/rowDecoder/main_test.go +++ b/util/rowDecoder/main_test.go @@ -23,7 +23,7 @@ import ( func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), } testbridge.SetupForCommonTest() diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 0c9d1b926f657..6fee2c56bf16d 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -134,7 +134,7 @@ func (decoder *DatumMapDecoder) decodeColDatum(col *ColInfo, colData []byte) (ty case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: var t types.Time t.SetType(col.Ft.Tp) - t.SetFsp(int8(col.Ft.Decimal)) + t.SetFsp(col.Ft.Decimal) err := t.FromPackedUint(decodeUint(colData)) if err != nil { return d, err @@ -149,7 +149,7 @@ func (decoder *DatumMapDecoder) decodeColDatum(col *ColInfo, colData []byte) (ty case mysql.TypeDuration: var dur types.Duration dur.Duration = time.Duration(decodeInt(colData)) - dur.Fsp = int8(col.Ft.Decimal) + dur.Fsp = col.Ft.Decimal d.SetMysqlDuration(dur) case mysql.TypeEnum: // ignore error deliberately, to read empty enum value. @@ -309,7 +309,7 @@ func (decoder *ChunkDecoder) decodeColToChunk(colIdx int, col *ColInfo, colData case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: var t types.Time t.SetType(col.Ft.Tp) - t.SetFsp(int8(col.Ft.Decimal)) + t.SetFsp(col.Ft.Decimal) err := t.FromPackedUint(decodeUint(colData)) if err != nil { return err @@ -324,7 +324,7 @@ func (decoder *ChunkDecoder) decodeColToChunk(colIdx int, col *ColInfo, colData case mysql.TypeDuration: var dur types.Duration dur.Duration = time.Duration(decodeInt(colData)) - dur.Fsp = int8(col.Ft.Decimal) + dur.Fsp = col.Ft.Decimal chk.AppendDuration(colIdx, dur) case mysql.TypeEnum: // ignore error deliberately, to read empty enum value. diff --git a/util/rowcodec/row.go b/util/rowcodec/row.go index 2cafab6c875a9..0e6b1a9ede213 100644 --- a/util/rowcodec/row.go +++ b/util/rowcodec/row.go @@ -18,7 +18,7 @@ import ( "encoding/binary" ) -// row is the struct type used to access the a row. +// row is the struct type used to access a row. type row struct { // small: colID []byte, offsets []uint16, optimized for most cases. // large: colID []uint32, offsets []uint32. diff --git a/util/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index 1ea59e0eea564..316ade488e1a5 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -891,9 +891,9 @@ var ( return d } } - withLen = func(len int) func(d types.Datum) types.Datum { + withLen = func(l int) func(d types.Datum) types.Datum { return func(d types.Datum) types.Datum { - d.SetLength(len) + d.SetLength(l) return d } } diff --git a/util/selection/selection.go b/util/selection/selection.go index 987e798d77096..7e72c43eee815 100644 --- a/util/selection/selection.go +++ b/util/selection/selection.go @@ -52,6 +52,8 @@ func introselect(data Interface, left, right, k int, depth int) int { } } +// quickselect is used in test for comparison. +// nolint: unused func quickselect(data Interface, left, right, k int) int { if left == right { return left diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index 8ca1fbf67f918..45f283e7f5e78 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -45,43 +45,67 @@ type RestrictedSQLExecutor interface { // Attention: it does not prevent you from doing parse("select '%?", ";SQL injection!;") => "select '';SQL injection!;'". // One argument should be a standalone entity. It should not "concat" with other placeholders and characters. // This function only saves you from processing potentially unsafe parameters. - ParseWithParams(ctx context.Context, forceUTF8SQL bool, sql string, args ...interface{}) (ast.StmtNode, error) - // ExecRestrictedStmt run sql statement in ctx with some restriction. + ParseWithParams(ctx context.Context, sql string, args ...interface{}) (ast.StmtNode, error) + // ExecRestrictedStmt run sql statement in ctx with some restrictions. ExecRestrictedStmt(ctx context.Context, stmt ast.StmtNode, opts ...OptionFuncAlias) ([]chunk.Row, []*ast.ResultField, error) + // ExecRestrictedSQL run sql string in ctx with internal session. + ExecRestrictedSQL(ctx context.Context, opts []OptionFuncAlias, sql string, args ...interface{}) ([]chunk.Row, []*ast.ResultField, error) } -// ExecOption is a struct defined for ExecRestrictedStmt option. +// ExecOption is a struct defined for ExecRestrictedStmt/SQL option. type ExecOption struct { IgnoreWarning bool SnapshotTS uint64 AnalyzeVer int + UseCurSession bool } -// OptionFuncAlias is defined for the optional paramater of ExecRestrictedStmt. +// OptionFuncAlias is defined for the optional parameter of ExecRestrictedStmt/SQL. type OptionFuncAlias = func(option *ExecOption) -// ExecOptionIgnoreWarning tells ExecRestrictedStmt to ignore the warnings. +// ExecOptionIgnoreWarning tells ExecRestrictedStmt/SQL to ignore the warnings. var ExecOptionIgnoreWarning OptionFuncAlias = func(option *ExecOption) { option.IgnoreWarning = true } -// ExecOptionAnalyzeVer1 tells ExecRestrictedStmt to collect statistics with version1. +// ExecOptionAnalyzeVer1 tells ExecRestrictedStmt/SQL to collect statistics with version1. var ExecOptionAnalyzeVer1 OptionFuncAlias = func(option *ExecOption) { option.AnalyzeVer = 1 } +// ExecOptionAnalyzeVer2 tells ExecRestrictedStmt/SQL to collect statistics with version2. // ExecOptionAnalyzeVer2 tells ExecRestrictedStmt to collect statistics with version2. var ExecOptionAnalyzeVer2 OptionFuncAlias = func(option *ExecOption) { option.AnalyzeVer = 2 } -// ExecOptionWithSnapshot tells ExecRestrictedStmt to use a snapshot. +// ExecOptionUseCurSession tells ExecRestrictedStmt/SQL to use current session. +var ExecOptionUseCurSession OptionFuncAlias = func(option *ExecOption) { + option.UseCurSession = true +} + +// ExecOptionUseSessionPool tells ExecRestrictedStmt/SQL to use session pool. +// UseCurSession is false by default, sometimes we set it explicitly for readability +var ExecOptionUseSessionPool OptionFuncAlias = func(option *ExecOption) { + option.UseCurSession = false +} + +// ExecOptionWithSnapshot tells ExecRestrictedStmt/SQL to use a snapshot. func ExecOptionWithSnapshot(snapshot uint64) OptionFuncAlias { return func(option *ExecOption) { option.SnapshotTS = snapshot } } +// GetExecOption applies OptionFuncs and return ExecOption +func GetExecOption(opts []OptionFuncAlias) ExecOption { + var execOption ExecOption + for _, opt := range opts { + opt(&execOption) + } + return execOption +} + // SQLExecutor is an interface provides executing normal sql statement. // Why we need this interface? To break circle dependence of packages. // For example, privilege/privileges package need execute SQL, if it use diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index 3850abb486736..a0003d9390eef 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/set" @@ -40,29 +39,25 @@ type stmtSummaryReader struct { ssMap *stmtSummaryByDigestMap columnValueFactories []columnValueFactory checker *stmtSummaryChecker + tz *time.Location } // NewStmtSummaryReader return a new statement summaries reader. -func NewStmtSummaryReader(user *auth.UserIdentity, hasProcessPriv bool, cols []*model.ColumnInfo, instanceAddr string) *stmtSummaryReader { +func NewStmtSummaryReader(user *auth.UserIdentity, hasProcessPriv bool, cols []*model.ColumnInfo, instanceAddr string, tz *time.Location) *stmtSummaryReader { reader := &stmtSummaryReader{ user: user, hasProcessPriv: hasProcessPriv, columns: cols, instanceAddr: instanceAddr, ssMap: StmtSummaryByDigestMap, + tz: tz, } // initialize column value factories. reader.columnValueFactories = make([]columnValueFactory, len(reader.columns)) for i, col := range reader.columns { factory, ok := columnValueFactoryMap[col.Name.O] if !ok { - if col.Name.O == util.ClusterTableInstanceColumnName { - factory = func(ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { - return reader.instanceAddr - } - } else { - panic(fmt.Sprintf("should never happen, should register new column %v into columnValueFactoryMap", col.Name.O)) - } + panic(fmt.Sprintf("should never happen, should register new column %v into columnValueFactoryMap", col.Name.O)) } reader.columnValueFactories[i] = factory } @@ -108,11 +103,7 @@ func (ssr *stmtSummaryReader) GetStmtSummaryHistoryRows() [][]types.Datum { historySize := ssMap.historySize() rows := make([][]types.Datum, 0, len(values)*historySize) for _, value := range values { - ssbd := value.(*stmtSummaryByDigest) - if ssr.checker != nil && !ssr.checker.isDigestValid(ssbd.digest) { - continue - } - records := ssr.getStmtByDigestHistoryRow(ssbd, historySize) + records := ssr.getStmtByDigestHistoryRow(value.(*stmtSummaryByDigest), historySize) rows = append(rows, records...) } @@ -153,14 +144,14 @@ func (ssr *stmtSummaryReader) getStmtByDigestElementRow(ssElement *stmtSummaryBy defer ssElement.Unlock() datums := make([]types.Datum, len(ssr.columnValueFactories)) for i, factory := range ssr.columnValueFactories { - datums[i] = types.NewDatum(factory(ssElement, ssbd)) + datums[i] = types.NewDatum(factory(ssr, ssElement, ssbd)) } return datums } func (ssr *stmtSummaryReader) getStmtByDigestHistoryRow(ssbd *stmtSummaryByDigest, historySize int) [][]types.Datum { // Collect all history summaries to an array. - ssElements := ssbd.collectHistorySummaries(historySize) + ssElements := ssbd.collectHistorySummaries(ssr.checker, historySize) rows := make([][]types.Datum, 0, len(ssElements)) for _, ssElement := range ssElements { @@ -222,127 +213,139 @@ func (ssc *stmtSummaryChecker) isDigestValid(digest string) bool { // Statements summary table column name. const ( - SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" - SummaryEndTimeStr = "SUMMARY_END_TIME" - StmtTypeStr = "STMT_TYPE" - SchemaNameStr = "SCHEMA_NAME" - DigestStr = "DIGEST" - DigestTextStr = "DIGEST_TEXT" - TableNamesStr = "TABLE_NAMES" - IndexNamesStr = "INDEX_NAMES" - SampleUserStr = "SAMPLE_USER" - ExecCountStr = "EXEC_COUNT" - SumErrorsStr = "SUM_ERRORS" - SumWarningsStr = "SUM_WARNINGS" - SumLatencyStr = "SUM_LATENCY" - MaxLatencyStr = "MAX_LATENCY" - MinLatencyStr = "MIN_LATENCY" - AvgLatencyStr = "AVG_LATENCY" - AvgParseLatencyStr = "AVG_PARSE_LATENCY" - MaxParseLatencyStr = "MAX_PARSE_LATENCY" - AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" - MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" - SumCopTaskNumStr = "SUM_COP_TASK_NUM" - MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" - MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" - MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 - MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 - AvgProcessTimeStr = "AVG_PROCESS_TIME" - MaxProcessTimeStr = "MAX_PROCESS_TIME" - AvgWaitTimeStr = "AVG_WAIT_TIME" - MaxWaitTimeStr = "MAX_WAIT_TIME" - AvgBackoffTimeStr = "AVG_BACKOFF_TIME" - MaxBackoffTimeStr = "MAX_BACKOFF_TIME" - AvgTotalKeysStr = "AVG_TOTAL_KEYS" - MaxTotalKeysStr = "MAX_TOTAL_KEYS" - AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" - MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" - AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" - MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" - AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" - MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" - AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" - AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" - MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" - AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" - MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" - AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" - MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" - AvgCommitTimeStr = "AVG_COMMIT_TIME" - MaxCommitTimeStr = "MAX_COMMIT_TIME" - AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" - MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" - AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" - MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" - AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" - MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" - AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" - MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" - AvgWriteKeysStr = "AVG_WRITE_KEYS" - MaxWriteKeysStr = "MAX_WRITE_KEYS" - AvgWriteSizeStr = "AVG_WRITE_SIZE" - MaxWriteSizeStr = "MAX_WRITE_SIZE" - AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" - MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" - AvgTxnRetryStr = "AVG_TXN_RETRY" - MaxTxnRetryStr = "MAX_TXN_RETRY" - SumExecRetryStr = "SUM_EXEC_RETRY" - SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" - SumBackoffTimesStr = "SUM_BACKOFF_TIMES" - BackoffTypesStr = "BACKOFF_TYPES" - AvgMemStr = "AVG_MEM" - MaxMemStr = "MAX_MEM" - AvgDiskStr = "AVG_DISK" - MaxDiskStr = "MAX_DISK" - AvgKvTimeStr = "AVG_KV_TIME" - AvgPdTimeStr = "AVG_PD_TIME" - AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" - AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" - MaxResultRowsStr = "MAX_RESULT_ROWS" - MinResultRowsStr = "MIN_RESULT_ROWS" - AvgResultRowsStr = "AVG_RESULT_ROWS" - PreparedStr = "PREPARED" - AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" - FirstSeenStr = "FIRST_SEEN" - LastSeenStr = "LAST_SEEN" - PlanInCacheStr = "PLAN_IN_CACHE" - PlanCacheHitsStr = "PLAN_CACHE_HITS" - PlanInBindingStr = "PLAN_IN_BINDING" - QuerySampleTextStr = "QUERY_SAMPLE_TEXT" - PrevSampleTextStr = "PREV_SAMPLE_TEXT" - PlanDigestStr = "PLAN_DIGEST" - PlanStr = "PLAN" + ClusterTableInstanceColumnNameStr = "INSTANCE" + SummaryBeginTimeStr = "SUMMARY_BEGIN_TIME" + SummaryEndTimeStr = "SUMMARY_END_TIME" + StmtTypeStr = "STMT_TYPE" + SchemaNameStr = "SCHEMA_NAME" + DigestStr = "DIGEST" + DigestTextStr = "DIGEST_TEXT" + TableNamesStr = "TABLE_NAMES" + IndexNamesStr = "INDEX_NAMES" + SampleUserStr = "SAMPLE_USER" + ExecCountStr = "EXEC_COUNT" + SumErrorsStr = "SUM_ERRORS" + SumWarningsStr = "SUM_WARNINGS" + SumLatencyStr = "SUM_LATENCY" + MaxLatencyStr = "MAX_LATENCY" + MinLatencyStr = "MIN_LATENCY" + AvgLatencyStr = "AVG_LATENCY" + AvgParseLatencyStr = "AVG_PARSE_LATENCY" + MaxParseLatencyStr = "MAX_PARSE_LATENCY" + AvgCompileLatencyStr = "AVG_COMPILE_LATENCY" + MaxCompileLatencyStr = "MAX_COMPILE_LATENCY" + SumCopTaskNumStr = "SUM_COP_TASK_NUM" + MaxCopProcessTimeStr = "MAX_COP_PROCESS_TIME" + MaxCopProcessAddressStr = "MAX_COP_PROCESS_ADDRESS" + MaxCopWaitTimeStr = "MAX_COP_WAIT_TIME" // #nosec G101 + MaxCopWaitAddressStr = "MAX_COP_WAIT_ADDRESS" // #nosec G101 + AvgProcessTimeStr = "AVG_PROCESS_TIME" + MaxProcessTimeStr = "MAX_PROCESS_TIME" + AvgWaitTimeStr = "AVG_WAIT_TIME" + MaxWaitTimeStr = "MAX_WAIT_TIME" + AvgBackoffTimeStr = "AVG_BACKOFF_TIME" + MaxBackoffTimeStr = "MAX_BACKOFF_TIME" + AvgTotalKeysStr = "AVG_TOTAL_KEYS" + MaxTotalKeysStr = "MAX_TOTAL_KEYS" + AvgProcessedKeysStr = "AVG_PROCESSED_KEYS" + MaxProcessedKeysStr = "MAX_PROCESSED_KEYS" + AvgRocksdbDeleteSkippedCountStr = "AVG_ROCKSDB_DELETE_SKIPPED_COUNT" + MaxRocksdbDeleteSkippedCountStr = "MAX_ROCKSDB_DELETE_SKIPPED_COUNT" + AvgRocksdbKeySkippedCountStr = "AVG_ROCKSDB_KEY_SKIPPED_COUNT" + MaxRocksdbKeySkippedCountStr = "MAX_ROCKSDB_KEY_SKIPPED_COUNT" + AvgRocksdbBlockCacheHitCountStr = "AVG_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + MaxRocksdbBlockCacheHitCountStr = "MAX_ROCKSDB_BLOCK_CACHE_HIT_COUNT" + AvgRocksdbBlockReadCountStr = "AVG_ROCKSDB_BLOCK_READ_COUNT" + MaxRocksdbBlockReadCountStr = "MAX_ROCKSDB_BLOCK_READ_COUNT" + AvgRocksdbBlockReadByteStr = "AVG_ROCKSDB_BLOCK_READ_BYTE" + MaxRocksdbBlockReadByteStr = "MAX_ROCKSDB_BLOCK_READ_BYTE" + AvgPrewriteTimeStr = "AVG_PREWRITE_TIME" + MaxPrewriteTimeStr = "MAX_PREWRITE_TIME" + AvgCommitTimeStr = "AVG_COMMIT_TIME" + MaxCommitTimeStr = "MAX_COMMIT_TIME" + AvgGetCommitTsTimeStr = "AVG_GET_COMMIT_TS_TIME" + MaxGetCommitTsTimeStr = "MAX_GET_COMMIT_TS_TIME" + AvgCommitBackoffTimeStr = "AVG_COMMIT_BACKOFF_TIME" + MaxCommitBackoffTimeStr = "MAX_COMMIT_BACKOFF_TIME" + AvgResolveLockTimeStr = "AVG_RESOLVE_LOCK_TIME" + MaxResolveLockTimeStr = "MAX_RESOLVE_LOCK_TIME" + AvgLocalLatchWaitTimeStr = "AVG_LOCAL_LATCH_WAIT_TIME" + MaxLocalLatchWaitTimeStr = "MAX_LOCAL_LATCH_WAIT_TIME" + AvgWriteKeysStr = "AVG_WRITE_KEYS" + MaxWriteKeysStr = "MAX_WRITE_KEYS" + AvgWriteSizeStr = "AVG_WRITE_SIZE" + MaxWriteSizeStr = "MAX_WRITE_SIZE" + AvgPrewriteRegionsStr = "AVG_PREWRITE_REGIONS" + MaxPrewriteRegionsStr = "MAX_PREWRITE_REGIONS" + AvgTxnRetryStr = "AVG_TXN_RETRY" + MaxTxnRetryStr = "MAX_TXN_RETRY" + SumExecRetryStr = "SUM_EXEC_RETRY" + SumExecRetryTimeStr = "SUM_EXEC_RETRY_TIME" + SumBackoffTimesStr = "SUM_BACKOFF_TIMES" + BackoffTypesStr = "BACKOFF_TYPES" + AvgMemStr = "AVG_MEM" + MaxMemStr = "MAX_MEM" + AvgDiskStr = "AVG_DISK" + MaxDiskStr = "MAX_DISK" + AvgKvTimeStr = "AVG_KV_TIME" + AvgPdTimeStr = "AVG_PD_TIME" + AvgBackoffTotalTimeStr = "AVG_BACKOFF_TOTAL_TIME" + AvgWriteSQLRespTimeStr = "AVG_WRITE_SQL_RESP_TIME" + MaxResultRowsStr = "MAX_RESULT_ROWS" + MinResultRowsStr = "MIN_RESULT_ROWS" + AvgResultRowsStr = "AVG_RESULT_ROWS" + PreparedStr = "PREPARED" + AvgAffectedRowsStr = "AVG_AFFECTED_ROWS" + FirstSeenStr = "FIRST_SEEN" + LastSeenStr = "LAST_SEEN" + PlanInCacheStr = "PLAN_IN_CACHE" + PlanCacheHitsStr = "PLAN_CACHE_HITS" + PlanInBindingStr = "PLAN_IN_BINDING" + QuerySampleTextStr = "QUERY_SAMPLE_TEXT" + PrevSampleTextStr = "PREV_SAMPLE_TEXT" + PlanDigestStr = "PLAN_DIGEST" + PlanStr = "PLAN" ) -type columnValueFactory func(ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} +type columnValueFactory func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} var columnValueFactoryMap = map[string]columnValueFactory{ - SummaryBeginTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { - return types.NewTime(types.FromGoTime(time.Unix(ssElement.beginTime, 0)), mysql.TypeTimestamp, 0) + ClusterTableInstanceColumnNameStr: func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + return reader.instanceAddr }, - SummaryEndTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { - return types.NewTime(types.FromGoTime(time.Unix(ssElement.endTime, 0)), mysql.TypeTimestamp, 0) + SummaryBeginTimeStr: func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + beginTime := time.Unix(ssElement.beginTime, 0) + if beginTime.Location() != reader.tz { + beginTime = beginTime.In(reader.tz) + } + return types.NewTime(types.FromGoTime(beginTime), mysql.TypeTimestamp, 0) }, - StmtTypeStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + SummaryEndTimeStr: func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + endTime := time.Unix(ssElement.endTime, 0) + if endTime.Location() != reader.tz { + endTime = endTime.In(reader.tz) + } + return types.NewTime(types.FromGoTime(endTime), mysql.TypeTimestamp, 0) + }, + StmtTypeStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return ssbd.stmtType }, - SchemaNameStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + SchemaNameStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return convertEmptyToNil(ssbd.schemaName) }, - DigestStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + DigestStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return convertEmptyToNil(ssbd.digest) }, - DigestTextStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + DigestTextStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return ssbd.normalizedSQL }, - TableNamesStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + TableNamesStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return convertEmptyToNil(ssbd.tableNames) }, - IndexNamesStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + IndexNamesStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return convertEmptyToNil(strings.Join(ssElement.indexNames, ",")) }, - SampleUserStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SampleUserStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { sampleUser := "" for key := range ssElement.authUsers { sampleUser = key @@ -350,250 +353,258 @@ var columnValueFactoryMap = map[string]columnValueFactory{ } return convertEmptyToNil(sampleUser) }, - ExecCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + ExecCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.execCount }, - SumErrorsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumErrorsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sumErrors }, - SumWarningsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumWarningsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sumWarnings }, - SumLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.sumLatency) }, - MaxLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxLatency) }, - MinLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MinLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.minLatency) }, - AvgLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumLatency), ssElement.execCount) }, - AvgParseLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgParseLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumParseLatency), ssElement.execCount) }, - MaxParseLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxParseLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxParseLatency) }, - AvgCompileLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgCompileLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumCompileLatency), ssElement.execCount) }, - MaxCompileLatencyStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCompileLatencyStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxCompileLatency) }, - SumCopTaskNumStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumCopTaskNumStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sumNumCopTasks }, - MaxCopProcessTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCopProcessTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxCopProcessTime) }, - MaxCopProcessAddressStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCopProcessAddressStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return convertEmptyToNil(ssElement.maxCopProcessAddress) }, - MaxCopWaitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCopWaitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxCopWaitTime) }, - MaxCopWaitAddressStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCopWaitAddressStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return convertEmptyToNil(ssElement.maxCopWaitAddress) }, - AvgProcessTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgProcessTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumProcessTime), ssElement.execCount) }, - MaxProcessTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxProcessTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxProcessTime) }, - AvgWaitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgWaitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumWaitTime), ssElement.execCount) }, - MaxWaitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxWaitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxWaitTime) }, - AvgBackoffTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgBackoffTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumBackoffTime), ssElement.execCount) }, - MaxBackoffTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxBackoffTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxBackoffTime) }, - AvgTotalKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgTotalKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumTotalKeys, ssElement.execCount) }, - MaxTotalKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxTotalKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxTotalKeys }, - AvgProcessedKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgProcessedKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumProcessedKeys, ssElement.execCount) }, - MaxProcessedKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxProcessedKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxProcessedKeys }, - AvgRocksdbDeleteSkippedCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgRocksdbDeleteSkippedCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumRocksdbDeleteSkippedCount), ssElement.execCount) }, - MaxRocksdbDeleteSkippedCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxRocksdbDeleteSkippedCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxRocksdbDeleteSkippedCount }, - AvgRocksdbKeySkippedCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgRocksdbKeySkippedCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumRocksdbKeySkippedCount), ssElement.execCount) }, - MaxRocksdbKeySkippedCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxRocksdbKeySkippedCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxRocksdbKeySkippedCount }, - AvgRocksdbBlockCacheHitCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgRocksdbBlockCacheHitCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumRocksdbBlockCacheHitCount), ssElement.execCount) }, - MaxRocksdbBlockCacheHitCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxRocksdbBlockCacheHitCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxRocksdbBlockCacheHitCount }, - AvgRocksdbBlockReadCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgRocksdbBlockReadCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumRocksdbBlockReadCount), ssElement.execCount) }, - MaxRocksdbBlockReadCountStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxRocksdbBlockReadCountStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxRocksdbBlockReadCount }, - AvgRocksdbBlockReadByteStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgRocksdbBlockReadByteStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumRocksdbBlockReadByte), ssElement.execCount) }, - MaxRocksdbBlockReadByteStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxRocksdbBlockReadByteStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxRocksdbBlockReadByte }, - AvgPrewriteTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgPrewriteTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumPrewriteTime), ssElement.commitCount) }, - MaxPrewriteTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxPrewriteTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxPrewriteTime) }, - AvgCommitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgCommitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumCommitTime), ssElement.commitCount) }, - MaxCommitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCommitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxCommitTime) }, - AvgGetCommitTsTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgGetCommitTsTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumGetCommitTsTime), ssElement.commitCount) }, - MaxGetCommitTsTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxGetCommitTsTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxGetCommitTsTime) }, - AvgCommitBackoffTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgCommitBackoffTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumCommitBackoffTime, ssElement.commitCount) }, - MaxCommitBackoffTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxCommitBackoffTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxCommitBackoffTime }, - AvgResolveLockTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgResolveLockTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumResolveLockTime, ssElement.commitCount) }, - MaxResolveLockTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxResolveLockTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxResolveLockTime }, - AvgLocalLatchWaitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgLocalLatchWaitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumLocalLatchTime), ssElement.commitCount) }, - MaxLocalLatchWaitTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxLocalLatchWaitTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.maxLocalLatchTime) }, - AvgWriteKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgWriteKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgFloat(ssElement.sumWriteKeys, ssElement.commitCount) }, - MaxWriteKeysStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxWriteKeysStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxWriteKeys }, - AvgWriteSizeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgWriteSizeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgFloat(ssElement.sumWriteSize, ssElement.commitCount) }, - MaxWriteSizeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxWriteSizeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxWriteSize }, - AvgPrewriteRegionsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgPrewriteRegionsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgFloat(ssElement.sumPrewriteRegionNum, ssElement.commitCount) }, - MaxPrewriteRegionsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxPrewriteRegionsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int(ssElement.maxPrewriteRegionNum) }, - AvgTxnRetryStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgTxnRetryStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgFloat(ssElement.sumTxnRetry, ssElement.commitCount) }, - MaxTxnRetryStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxTxnRetryStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxTxnRetry }, - SumExecRetryStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumExecRetryStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int(ssElement.execRetryCount) }, - SumExecRetryTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumExecRetryTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return int64(ssElement.execRetryTime) }, - SumBackoffTimesStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + SumBackoffTimesStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sumBackoffTimes }, - BackoffTypesStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + BackoffTypesStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return formatBackoffTypes(ssElement.backoffTypes) }, - AvgMemStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgMemStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumMem, ssElement.execCount) }, - MaxMemStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxMemStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxMem }, - AvgDiskStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgDiskStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumDisk, ssElement.execCount) }, - MaxDiskStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxDiskStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxDisk }, - AvgKvTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgKvTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumKVTotal), ssElement.commitCount) }, - AvgPdTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgPdTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumPDTotal), ssElement.commitCount) }, - AvgBackoffTotalTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgBackoffTotalTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumBackoffTotal), ssElement.commitCount) }, - AvgWriteSQLRespTimeStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgWriteSQLRespTimeStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(int64(ssElement.sumWriteSQLRespTotal), ssElement.commitCount) }, - MaxResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MaxResultRowsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.maxResultRows }, - MinResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + MinResultRowsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.minResultRows }, - AvgResultRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgResultRowsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgInt(ssElement.sumResultRows, ssElement.execCount) }, - PreparedStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PreparedStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.prepared }, - AvgAffectedRowsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + AvgAffectedRowsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return avgFloat(int64(ssElement.sumAffectedRows), ssElement.execCount) }, - FirstSeenStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { - return types.NewTime(types.FromGoTime(ssElement.firstSeen), mysql.TypeTimestamp, 0) + FirstSeenStr: func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + firstSeen := ssElement.firstSeen + if firstSeen.Location() != reader.tz { + firstSeen = firstSeen.In(reader.tz) + } + return types.NewTime(types.FromGoTime(firstSeen), mysql.TypeTimestamp, 0) }, - LastSeenStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { - return types.NewTime(types.FromGoTime(ssElement.lastSeen), mysql.TypeTimestamp, 0) + LastSeenStr: func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + lastSeen := ssElement.lastSeen + if lastSeen.Location() != reader.tz { + lastSeen = lastSeen.In(reader.tz) + } + return types.NewTime(types.FromGoTime(lastSeen), mysql.TypeTimestamp, 0) }, - PlanInCacheStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PlanInCacheStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.planInCache }, - PlanCacheHitsStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PlanCacheHitsStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.planCacheHits }, - PlanInBindingStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PlanInBindingStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.planInBinding }, - QuerySampleTextStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + QuerySampleTextStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sampleSQL }, - PrevSampleTextStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PrevSampleTextStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.prevSQL }, - PlanDigestStr: func(_ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { + PlanDigestStr: func(_ *stmtSummaryReader, _ *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} { return ssbd.planDigest }, - PlanStr: func(ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + PlanStr: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { plan, err := plancodec.DecodePlan(ssElement.samplePlan) if err != nil { logutil.BgLogger().Error("decode plan in statement summary failed", zap.String("plan", ssElement.samplePlan), zap.String("query", ssElement.sampleSQL), zap.Error(err)) diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index f7760fe4198ab..9271f5c274ace 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -374,6 +374,7 @@ type BindableStmt struct { PlanHint string Charset string Collation string + Users map[string]struct{} // which users have processed this stmt } // GetMoreThanCntBindableStmt gets users' select/update/delete SQLs that occurred more than the specified count. @@ -401,6 +402,7 @@ func (ssMap *stmtSummaryByDigestMap) GetMoreThanCntBindableStmt(cnt int64) []*Bi PlanHint: ssElement.planHint, Charset: ssElement.charset, Collation: ssElement.collation, + Users: ssElement.authUsers, } // If it is SQL command prepare / execute, the ssElement.sampleSQL is `execute ...`, we should get the original select query. // If it is binary protocol prepare / execute, ssbd.normalizedSQL should be same as ssElement.sampleSQL. @@ -480,6 +482,7 @@ func (ssMap *stmtSummaryByDigestMap) SetMaxStmtCount(value uint) error { } // Used by tests +// nolint: unused func (ssMap *stmtSummaryByDigestMap) maxStmtCount() int { return int(ssMap.optMaxStmtCount.Load()) } @@ -571,13 +574,17 @@ func (ssbd *stmtSummaryByDigest) add(sei *StmtExecInfo, beginTime int64, interva } // collectHistorySummaries puts at most `historySize` summaries to an array. -func (ssbd *stmtSummaryByDigest) collectHistorySummaries(historySize int) []*stmtSummaryByDigestElement { +func (ssbd *stmtSummaryByDigest) collectHistorySummaries(checker *stmtSummaryChecker, historySize int) []*stmtSummaryByDigestElement { ssbd.Lock() defer ssbd.Unlock() if !ssbd.initialized { return nil } + if checker != nil && !checker.isDigestValid(ssbd.digest) { + return nil + } + ssElements := make([]*stmtSummaryByDigestElement, 0, ssbd.history.Len()) for listElement := ssbd.history.Front(); listElement != nil && len(ssElements) < historySize; listElement = listElement.Next() { ssElement := listElement.Value.(*stmtSummaryByDigestElement) diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index c3c3d8b7bcf0d..a84eb8799af07 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/plancodec" "github.com/stretchr/testify/require" @@ -740,7 +741,7 @@ func newStmtSummaryReaderForTest(ssMap *stmtSummaryByDigestMap) *stmtSummaryRead Offset: i, } } - reader := NewStmtSummaryReader(nil, true, cols, "") + reader := NewStmtSummaryReader(nil, true, cols, "", time.UTC) reader.ssMap = ssMap return reader } @@ -757,8 +758,8 @@ func TestToDatum(t *testing.T) { reader := newStmtSummaryReaderForTest(ssMap) datums := reader.GetStmtSummaryCurrentRows() require.Equal(t, 1, len(datums)) - n := types.NewTime(types.FromGoTime(time.Unix(ssMap.beginTimeForCurInterval, 0)), mysql.TypeTimestamp, types.DefaultFsp) - e := types.NewTime(types.FromGoTime(time.Unix(ssMap.beginTimeForCurInterval+1800, 0)), mysql.TypeTimestamp, types.DefaultFsp) + n := types.NewTime(types.FromGoTime(time.Unix(ssMap.beginTimeForCurInterval, 0).In(time.UTC)), mysql.TypeTimestamp, types.DefaultFsp) + e := types.NewTime(types.FromGoTime(time.Unix(ssMap.beginTimeForCurInterval+1800, 0).In(time.UTC)), mysql.TypeTimestamp, types.DefaultFsp) f := types.NewTime(types.FromGoTime(stmtExecInfo1.StartTime), mysql.TypeTimestamp, types.DefaultFsp) stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() expectedDatum := []interface{}{n, e, "Select", stmtExecInfo1.SchemaName, stmtExecInfo1.Digest, stmtExecInfo1.NormalizedSQL, @@ -988,11 +989,9 @@ func TestSetMaxStmtCountParallel(t *testing.T) { threads := 8 loops := 20 - wg := sync.WaitGroup{} - wg.Add(threads + 1) + var wg tidbutil.WaitGroupWrapper addStmtFunc := func() { - defer wg.Done() stmtExecInfo1 := generateAnyExecInfo() // Add 32 times with different digest. @@ -1002,24 +1001,26 @@ func TestSetMaxStmtCountParallel(t *testing.T) { } } for i := 0; i < threads; i++ { - go addStmtFunc() + wg.Run(addStmtFunc) } defer func() { - require.Nil(t, ssMap.SetMaxStmtCount(3000)) + require.NoError(t, ssMap.SetMaxStmtCount(3000)) }() setStmtCountFunc := func() { - defer wg.Done() // Turn down MaxStmtCount one by one. for i := 10; i > 0; i-- { - require.Nil(t, ssMap.SetMaxStmtCount(uint(i))) + require.NoError(t, ssMap.SetMaxStmtCount(uint(i))) } } - go setStmtCountFunc() + wg.Run(setStmtCountFunc) wg.Wait() + // add stmt again to make sure evict occurs after SetMaxStmtCount. + addStmtFunc() + reader := newStmtSummaryReaderForTest(ssMap) datums := reader.GetStmtSummaryCurrentRows() // due to evictions happened in cache, an additional record will be appended to the table. diff --git a/util/testleak/leaktest.go b/util/testleak/leaktest.go index a34ed5b01f5b1..13bb442f29293 100644 --- a/util/testleak/leaktest.go +++ b/util/testleak/leaktest.go @@ -61,7 +61,7 @@ func interestingGoroutines() (gs []string) { "github.com/pingcap/goleveldb/leveldb/util.(*BufferPool).drain", "github.com/pingcap/goleveldb/leveldb.(*DB).compactionError", "github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain", - "go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop", + "go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop", "go.etcd.io/etcd/v3/pkg/logutil.(*MergeLogger).outputLoop", "oracles.(*pdOracle).updateTS", "tikv.(*KVStore).runSafePointChecker", diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index e081f0b45849c..816a8bd4c7e18 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -93,11 +93,11 @@ var DatumEquals check.Checker = &datumEqualsChecker{ &check.CheckerInfo{Name: "DatumEquals", Params: []string{"obtained", "expected"}}, } -func (checker *datumEqualsChecker) Check(params []interface{}, names []string) (result bool, error string) { +func (checker *datumEqualsChecker) Check(params []interface{}, names []string) (result bool, errStr string) { defer func() { if v := recover(); v != nil { result = false - error = fmt.Sprint(v) + errStr = fmt.Sprint(v) logutil.BgLogger().Error("panic in datumEqualsChecker.Check", zap.Reflect("r", v), zap.Stack("stack trace")) @@ -208,7 +208,7 @@ var HandleEquals = &handleEqualsChecker{ &check.CheckerInfo{Name: "HandleEquals", Params: []string{"obtained", "expected"}}, } -func (checker *handleEqualsChecker) Check(params []interface{}, names []string) (result bool, error string) { +func (checker *handleEqualsChecker) Check(params []interface{}, names []string) (result bool, errStr string) { if params[0] == nil && params[1] == nil { return true, "" } diff --git a/util/topsql/main_test.go b/util/topsql/main_test.go index 65a61a3e6d727..9de0c20eba3a8 100644 --- a/util/topsql/main_test.go +++ b/util/topsql/main_test.go @@ -23,12 +23,5 @@ import ( func TestMain(m *testing.M) { testbridge.SetupForCommonTest() - - opts := []goleak.Option{ - goleak.IgnoreTopFunction("time.Sleep"), - goleak.IgnoreTopFunction("runtime/pprof.readProfile"), - goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), - } - - goleak.VerifyTestMain(m, opts...) + goleak.VerifyTestMain(m) } diff --git a/util/topsql/reporter/mock/server.go b/util/topsql/reporter/mock/server.go index 59de46443c6e4..7f089230c134e 100644 --- a/util/topsql/reporter/mock/server.go +++ b/util/topsql/reporter/mock/server.go @@ -135,11 +135,20 @@ func (svr *mockAgentServer) ReportPlanMeta(stream tipb.TopSQLAgent_ReportPlanMet return stream.SendAndClose(&tipb.EmptyResponse{}) } -func (svr *mockAgentServer) WaitCollectCnt(cnt int, timeout time.Duration) { - start := time.Now() +func (svr *mockAgentServer) RecordsCnt() int { svr.Lock() - old := len(svr.records) - svr.Unlock() + defer svr.Unlock() + return len(svr.records) +} + +func (svr *mockAgentServer) SQLMetaCnt() int { + svr.Lock() + defer svr.Unlock() + return len(svr.sqlMetas) +} + +func (svr *mockAgentServer) WaitCollectCnt(old, cnt int, timeout time.Duration) { + start := time.Now() for { svr.Lock() if len(svr.records)-old >= cnt { @@ -154,6 +163,22 @@ func (svr *mockAgentServer) WaitCollectCnt(cnt int, timeout time.Duration) { } } +func (svr *mockAgentServer) WaitCollectCntOfSQLMeta(old, cnt int, timeout time.Duration) { + start := time.Now() + for { + svr.Lock() + if len(svr.sqlMetas)-old >= cnt { + svr.Unlock() + return + } + svr.Unlock() + if time.Since(start) > timeout { + return + } + time.Sleep(time.Millisecond) + } +} + func (svr *mockAgentServer) GetSQLMetaByDigestBlocking(digest []byte, timeout time.Duration) (meta tipb.SQLMeta, exist bool) { start := time.Now() for { diff --git a/util/topsql/reporter/single_target_test.go b/util/topsql/reporter/single_target_test.go index 6c9d5a193a2a6..f9f2010f6084e 100644 --- a/util/topsql/reporter/single_target_test.go +++ b/util/topsql/reporter/single_target_test.go @@ -43,6 +43,9 @@ func TestSingleTargetDataSink(t *testing.T) { ds.Start() defer ds.Close() + recordsCnt := server.RecordsCnt() + sqlMetaCnt := server.SQLMetaCnt() + err = ds.TrySend(&ReportData{ DataRecords: []tipb.TopSQLRecord{{ SqlDigest: []byte("S1"), @@ -66,7 +69,8 @@ func TestSingleTargetDataSink(t *testing.T) { }, time.Now().Add(10*time.Second)) assert.NoError(t, err) - server.WaitCollectCnt(1, 5*time.Second) + server.WaitCollectCnt(recordsCnt, 1, 5*time.Second) + server.WaitCollectCntOfSQLMeta(sqlMetaCnt, 1, 5*time.Second) assert.Len(t, server.GetLatestRecords(), 1) assert.Len(t, server.GetTotalSQLMetas(), 1) diff --git a/util/topsql/topsql_test.go b/util/topsql/topsql_test.go index 2678f5e2bf561..401ae47820ed8 100644 --- a/util/topsql/topsql_test.go +++ b/util/topsql/topsql_test.go @@ -109,6 +109,7 @@ func TestTopSQLReporter(t *testing.T) { defer func() { ds.Close() report.Close() + server.Stop() }() reqs := []struct { @@ -123,9 +124,9 @@ func TestTopSQLReporter(t *testing.T) { defer wg.Wait() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - sqlMap := make(map[string]string) sql2plan := make(map[string]string) + recordsCnt := server.RecordsCnt() for _, req := range reqs { sql2plan[req.sql] = req.plan sqlDigest := mock.GenSQLDigest(req.sql) @@ -142,29 +143,35 @@ func TestTopSQLReporter(t *testing.T) { } }) } - server.WaitCollectCnt(1, time.Second*5) - records := server.GetLatestRecords() checkSQLPlanMap := map[string]struct{}{} - for _, req := range records { - require.Greater(t, len(req.Items), 0) - require.Greater(t, req.Items[0].CpuTimeMs, uint32(0)) - sqlMeta, exist := server.GetSQLMetaByDigestBlocking(req.SqlDigest, time.Second) - require.True(t, exist) - expectedNormalizedSQL, exist := sqlMap[string(req.SqlDigest)] - require.True(t, exist) - require.Equal(t, expectedNormalizedSQL, sqlMeta.NormalizedSql) - - expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] - if expectedNormalizedPlan == "" || len(req.PlanDigest) == 0 { - require.Equal(t, len(req.PlanDigest), 0) - continue + for retry := 0; retry < 5; retry++ { + server.WaitCollectCnt(recordsCnt, 1, time.Second*5) + records := server.GetLatestRecords() + for _, req := range records { + require.Greater(t, len(req.Items), 0) + require.Greater(t, req.Items[0].CpuTimeMs, uint32(0)) + sqlMeta, exist := server.GetSQLMetaByDigestBlocking(req.SqlDigest, time.Second) + require.True(t, exist) + expectedNormalizedSQL, exist := sqlMap[string(req.SqlDigest)] + require.True(t, exist) + require.Equal(t, expectedNormalizedSQL, sqlMeta.NormalizedSql) + + expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] + if expectedNormalizedPlan == "" || len(req.PlanDigest) == 0 { + require.Len(t, req.PlanDigest, 0) + checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} + continue + } + normalizedPlan, exist := server.GetPlanMetaByDigestBlocking(req.PlanDigest, time.Second) + require.True(t, exist) + require.Equal(t, expectedNormalizedPlan, normalizedPlan) + checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} + } + if len(checkSQLPlanMap) == len(reqs) { + break } - normalizedPlan, exist := server.GetPlanMetaByDigestBlocking(req.PlanDigest, time.Second) - require.True(t, exist) - require.Equal(t, expectedNormalizedPlan, normalizedPlan) - checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} } - require.Equal(t, 2, len(checkSQLPlanMap)) + require.Equal(t, len(reqs), len(checkSQLPlanMap)) } func TestMaxSQLAndPlanTest(t *testing.T) { @@ -319,7 +326,7 @@ func TestTopSQLPubSub(t *testing.T) { expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] if expectedNormalizedPlan == "" || len(record.PlanDigest) == 0 { - require.Equal(t, len(record.PlanDigest), 0) + require.Len(t, record.PlanDigest, 0) continue } normalizedPlan, exist := planMetas[string(record.PlanDigest)] @@ -327,7 +334,7 @@ func TestTopSQLPubSub(t *testing.T) { require.Equal(t, expectedNormalizedPlan, normalizedPlan) checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} } - require.Equal(t, len(checkSQLPlanMap), 2) + require.Len(t, checkSQLPlanMap, 2) } func TestPubSubWhenReporterIsStopped(t *testing.T) { diff --git a/util/tracing/opt_trace.go b/util/tracing/opt_trace.go index 9977bddaa336c..6d7cae715cf0c 100644 --- a/util/tracing/opt_trace.go +++ b/util/tracing/opt_trace.go @@ -58,7 +58,7 @@ func (tracer *LogicalOptimizeTracer) AppendRuleTracerStepToCurrent(id int, tp, r // RecordFinalLogicalPlan add plan trace after logical optimize func (tracer *LogicalOptimizeTracer) RecordFinalLogicalPlan(final *PlanTrace) { - tracer.FinalLogicalPlan = toFlattenLogicalPlanTrace(final) + tracer.FinalLogicalPlan = toFlattenPlanTrace(final) tracer.removeUselessStep() } @@ -86,7 +86,7 @@ type LogicalRuleOptimizeTracer struct { func buildLogicalRuleOptimizeTracerBeforeOptimize(index int, name string, before *PlanTrace) *LogicalRuleOptimizeTracer { return &LogicalRuleOptimizeTracer{ Index: index, - Before: toFlattenLogicalPlanTrace(before), + Before: toFlattenPlanTrace(before), RuleName: name, Steps: make([]LogicalRuleOptimizeTraceStep, 0), } @@ -102,8 +102,8 @@ type LogicalRuleOptimizeTraceStep struct { Index int `json:"index"` } -// toFlattenLogicalPlanTrace transform LogicalPlanTrace into FlattenLogicalPlanTrace -func toFlattenLogicalPlanTrace(root *PlanTrace) []*PlanTrace { +// toFlattenPlanTrace transform plan into PlanTrace +func toFlattenPlanTrace(root *PlanTrace) []*PlanTrace { wrapper := &flattenWrapper{flatten: make([]*PlanTrace, 0)} flattenLogicalPlanTrace(root, wrapper) return wrapper.flatten @@ -168,7 +168,7 @@ type PhysicalOptimizeTracer struct { // RecordFinalPlanTrace records final physical plan trace func (tracer *PhysicalOptimizeTracer) RecordFinalPlanTrace(root *PlanTrace) { - tracer.Final = toFlattenLogicalPlanTrace(root) + tracer.Final = toFlattenPlanTrace(root) tracer.buildCandidatesInfo() } @@ -235,9 +235,17 @@ type OptimizeTracer struct { Physical *PhysicalOptimizeTracer `json:"physical"` // FinalPlan indicates the plan after post optimize FinalPlan []*PlanTrace `json:"final"` + // IsFastPlan indicates whether the plan is generated by fast plan + IsFastPlan bool `json:"isFastPlan"` +} + +// SetFastPlan sets fast plan +func (tracer *OptimizeTracer) SetFastPlan(final *PlanTrace) { + tracer.FinalPlan = toFlattenPlanTrace(final) + tracer.IsFastPlan = true } // RecordFinalPlan records plan after post optimize func (tracer *OptimizeTracer) RecordFinalPlan(final *PlanTrace) { - tracer.FinalPlan = toFlattenLogicalPlanTrace(final) + tracer.FinalPlan = toFlattenPlanTrace(final) } diff --git a/util/tracing/opt_trace_test.go b/util/tracing/opt_trace_test.go index f45355058eeba..acf8f82f6e895 100644 --- a/util/tracing/opt_trace_test.go +++ b/util/tracing/opt_trace_test.go @@ -100,6 +100,6 @@ func TestFlattenLogicalPlanTrace(t *testing.T) { ChildrenID: []int{2, 3}, }, } - require.EqualValues(t, toFlattenLogicalPlanTrace(root1), expect1) - require.EqualValues(t, toFlattenLogicalPlanTrace(root2), expect2) + require.EqualValues(t, toFlattenPlanTrace(root1), expect1) + require.EqualValues(t, toFlattenPlanTrace(root2), expect2) }